aboutsummaryrefslogtreecommitdiff
path: root/sql/core
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-14259] [SQL] Merging small files together based on the cost of openingDavies Liu2016-04-043-19/+21
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR basically re-do the things in #12068 but with a different model, which should work better in case of small files with different sizes. ## How was this patch tested? Updated existing tests. Ran a query on thousands of partitioned small files locally, with all default settings (the cost to open a file should be over estimated), the durations of tasks become smaller and smaller, which is good (the last few tasks will be shortest). Author: Davies Liu <davies@databricks.com> Closes #12095 from davies/file_cost.
* [SPARK-14334] [SQL] add toLocalIterator for Dataset/DataFrameDavies Liu2016-04-044-11/+60
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? RDD.toLocalIterator() could be used to fetch one partition at a time to reduce the memory usage. Right now, for Dataset/Dataframe we have to use df.rdd.toLocalIterator, which is super slow also requires lots of memory (because of the Java serializer or even Kyro serializer). This PR introduce an optimized toLocalIterator for Dataset/DataFrame, which is much faster and requires much less memory. For a partition with 5 millions rows, `df.rdd.toIterator` took about 100 seconds, but df.toIterator took less than 7 seconds. For 10 millions row, rdd.toIterator will crash (not enough memory) with 4G heap, but df.toLocalIterator could finished in 12 seconds. The JDBC server has been updated to use DataFrame.toIterator. ## How was this patch tested? Existing tests. Author: Davies Liu <davies@databricks.com> Closes #12114 from davies/local_iterator.
* [SPARK-12981] [SQL] extract Pyhton UDF in physical planDavies Liu2016-04-046-69/+55
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently we extract Python UDFs into a special logical plan EvaluatePython in analyzer, But EvaluatePython is not part of catalyst, many rules have no knowledge of it , which will break many things (for example, filter push down or column pruning). We should treat Python UDFs as normal expressions, until we want to evaluate in physical plan, we could extract them in end of optimizer, or physical plan. This PR extract Python UDFs in physical plan. Closes #10935 ## How was this patch tested? Added regression tests. Author: Davies Liu <davies@databricks.com> Closes #12127 from davies/py_udf.
* [SPARK-14176][SQL] Add DataFrameWriter.trigger to set the stream batch periodShixiong Zhu2016-04-049-13/+413
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add a processing time trigger to control the batch processing speed ## How was this patch tested? Unit tests Author: Shixiong Zhu <shixiong@databricks.com> Closes #11976 from zsxwing/trigger.
* [SPARK-14137] [SQL] Cleanup hash joinDavies Liu2016-04-046-401/+268
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR did a few cleanup on HashedRelation and HashJoin: 1) Merge HashedRelation and UniqueHashedRelation together 2) Return an iterator from HashedRelation, so we donot need a create many UnsafeRow objects. 3) Return a copy of HashedRelation for thread-safety in BroadcastJoin, so we can re-use the UnafeRow objects. 4) Cleanup HashJoin, share most of the code between BroadcastHashJoin and ShuffleHashJoin 5) Removed UniqueLongHashedRelation, which will be replaced by LongUnsafeMap (another PR). 6) Update benchmark, before this patch, the selectivity of joins are too high. ## How was this patch tested? Existing tests. Author: Davies Liu <davies@databricks.com> Closes #12102 from davies/cleanup_hash.
* [SPARK-14360][SQL] QueryExecution.debug.codegen() to dump codegenReynold Xin2016-04-041-0/+16
| | | | | | | | | | | | ## What changes were proposed in this pull request? We recently added the ability to dump the generated code for a given query. However, the method is only available through an implicit after an import. It'd slightly simplify things if it can be called directly in queryExecution. ## How was this patch tested? Manually tested in spark-shell. Author: Reynold Xin <rxin@databricks.com> Closes #12144 from rxin/SPARK-14360.
* [SPARK-14356] Update spark.sql.execution.debug to work on DatasetsMatei Zaharia2016-04-032-2/+8
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Update DebugQuery to work on Datasets of any type, not just DataFrames. ## How was this patch tested? Added unit tests, checked in spark-shell. Author: Matei Zaharia <matei@databricks.com> Closes #12140 from mateiz/debug-dataset.
* [SPARK-14355][BUILD] Fix typos in Exception/Testcase/Comments and static ↵Dongjoon Hyun2016-04-0319-25/+25
| | | | | | | | | | | | | | | | | | | | | analysis results ## What changes were proposed in this pull request? This PR contains the following 5 types of maintenance fix over 59 files (+94 lines, -93 lines). - Fix typos(exception/log strings, testcase name, comments) in 44 lines. - Fix lint-java errors (MaxLineLength) in 6 lines. (New codes after SPARK-14011) - Use diamond operators in 40 lines. (New codes after SPARK-13702) - Fix redundant semicolon in 5 lines. - Rename class `InferSchemaSuite` to `CSVInferSchemaSuite` in CSVInferSchemaSuite.scala. ## How was this patch tested? Manual and pass the Jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12139 from dongjoon-hyun/SPARK-14355.
* [SPARK-14350][SQL] EXPLAIN output should be in a single cellDongjoon Hyun2016-04-031-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? EXPLAIN output should be in a single cell. **Before** ``` scala> sql("explain select 1").collect() res0: Array[org.apache.spark.sql.Row] = Array([== Physical Plan ==], [WholeStageCodegen], [: +- Project [1 AS 1#1]], [: +- INPUT], [+- Scan OneRowRelation[]]) ``` **After** ``` scala> sql("explain select 1").collect() res1: Array[org.apache.spark.sql.Row] = Array([== Physical Plan == WholeStageCodegen : +- Project [1 AS 1#4] : +- INPUT +- Scan OneRowRelation[]]) ``` Or, ``` scala> sql("explain select 1").head res1: org.apache.spark.sql.Row = [== Physical Plan == WholeStageCodegen : +- Project [1 AS 1#5] : +- INPUT +- Scan OneRowRelation[]] ``` Please note that `Spark-shell(Scala-shell)` trims long string output. So, you may need to use `println` to get full strings. ``` scala> println(sql("explain codegen select 'a' as a group by 1").head) [Found 2 WholeStageCodegen subtrees. == Subtree 1 / 2 == WholeStageCodegen ... /* 059 */ } /* 060 */ } ] ``` ## How was this patch tested? Pass the Jenkins tests. (Testcases are updated.) Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12137 from dongjoon-hyun/SPARK-14350.
* [SPARK-14231] [SQL] JSON data source infers floating-point values as a ↵hyukjinkwon2016-04-025-12/+69
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | double when they do not fit in a decimal ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-14231 Currently, JSON data source supports to infer `DecimalType` for big numbers and `floatAsBigDecimal` option which reads floating-point values as `DecimalType`. But there are few restrictions in Spark `DecimalType` below: 1. The precision cannot be bigger than 38. 2. scale cannot be bigger than precision. Currently, both restrictions are not being handled. This PR handles the cases by inferring them as `DoubleType`. Also, the option name was changed from `floatAsBigDecimal` to `prefersDecimal` as suggested [here](https://issues.apache.org/jira/browse/SPARK-14231?focusedCommentId=15215579&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15215579). So, the codes below: ```scala def doubleRecords: RDD[String] = sqlContext.sparkContext.parallelize( s"""{"a": 1${"0" * 38}, "b": 0.01}""" :: s"""{"a": 2${"0" * 38}, "b": 0.02}""" :: Nil) val jsonDF = sqlContext.read .option("prefersDecimal", "true") .json(doubleRecords) jsonDF.printSchema() ``` produces below: - **Before** ```scala org.apache.spark.sql.AnalysisException: Decimal scale (2) cannot be greater than precision (1).; at org.apache.spark.sql.types.DecimalType.<init>(DecimalType.scala:44) at org.apache.spark.sql.execution.datasources.json.InferSchema$.org$apache$spark$sql$execution$datasources$json$InferSchema$$inferField(InferSchema.scala:144) at org.apache.spark.sql.execution.datasources.json.InferSchema$.org$apache$spark$sql$execution$datasources$json$InferSchema$$inferField(InferSchema.scala:108) at ... ``` - **After** ```scala root |-- a: double (nullable = true) |-- b: double (nullable = true) ``` ## How was this patch tested? Unit tests were used and `./dev/run_tests` for coding style tests. Author: hyukjinkwon <gurwls223@gmail.com> Closes #12030 from HyukjinKwon/SPARK-14231.
* [SPARK-13996] [SQL] Add more not null attributes for Filter codegenLiang-Chi Hsieh2016-04-021-4/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? JIRA: https://issues.apache.org/jira/browse/SPARK-13996 Filter codegen finds the attributes not null by checking IsNotNull(a) expression with a condition if child.output.contains(a). However, the current approach to checking it is not comprehensive. We can improve it. E.g., for this plan: val rdd = sqlContext.sparkContext.makeRDD(Seq(Row(1, "1"), Row(null, "1"), Row(2, "2"))) val schema = new StructType().add("k", IntegerType).add("v", StringType) val smallDF = sqlContext.createDataFrame(rdd, schema) val df = smallDF.filter("isnotnull(k + 1)") The code snippet generated without this patch: /* 031 */ protected void processNext() throws java.io.IOException { /* 032 */ /*** PRODUCE: Filter isnotnull((k#0 + 1)) */ /* 033 */ /* 034 */ /*** PRODUCE: INPUT */ /* 035 */ /* 036 */ while (!shouldStop() && inputadapter_input.hasNext()) { /* 037 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 038 */ /*** CONSUME: Filter isnotnull((k#0 + 1)) */ /* 039 */ /* input[0, int] */ /* 040 */ boolean filter_isNull = inputadapter_row.isNullAt(0); /* 041 */ int filter_value = filter_isNull ? -1 : (inputadapter_row.getInt(0)); /* 042 */ /* 043 */ /* isnotnull((input[0, int] + 1)) */ /* 044 */ /* (input[0, int] + 1) */ /* 045 */ boolean filter_isNull3 = true; /* 046 */ int filter_value3 = -1; /* 047 */ /* 048 */ if (!filter_isNull) { /* 049 */ filter_isNull3 = false; // resultCode could change nullability. /* 050 */ filter_value3 = filter_value + 1; /* 051 */ /* 052 */ } /* 053 */ if (!(!(filter_isNull3))) continue; /* 054 */ /* 055 */ filter_metricValue.add(1); With this patch: /* 031 */ protected void processNext() throws java.io.IOException { /* 032 */ /*** PRODUCE: Filter isnotnull((k#0 + 1)) */ /* 033 */ /* 034 */ /*** PRODUCE: INPUT */ /* 035 */ /* 036 */ while (!shouldStop() && inputadapter_input.hasNext()) { /* 037 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 038 */ /*** CONSUME: Filter isnotnull((k#0 + 1)) */ /* 039 */ /* input[0, int] */ /* 040 */ boolean filter_isNull = inputadapter_row.isNullAt(0); /* 041 */ int filter_value = filter_isNull ? -1 : (inputadapter_row.getInt(0)); /* 042 */ /* 043 */ if (filter_isNull) continue; /* 044 */ /* 045 */ filter_metricValue.add(1); ## How was this patch tested? Existing tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #11810 from viirya/add-more-not-null-attrs.
* [MINOR][DOCS] Use multi-line JavaDoc comments in Scala code.Dongjoon Hyun2016-04-0224-353/+361
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR aims to fix all Scala-Style multiline comments into Java-Style multiline comments in Scala codes. (All comment-only changes over 77 files: +786 lines, −747 lines) ## How was this patch tested? Manual. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12130 from dongjoon-hyun/use_multiine_javadoc_comments.
* [HOTFIX] Disable StateStoreSuite.maintenanceReynold Xin2016-04-021-1/+1
|
* [MINOR] Typo fixesJacek Laskowski2016-04-023-8/+8
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Typo fixes. No functional changes. ## How was this patch tested? Built the sources and ran with samples. Author: Jacek Laskowski <jacek@japila.pl> Closes #11802 from jaceklaskowski/typo-fixes.
* [HOTFIX] Fix compilation break.Reynold Xin2016-04-022-5/+4
|
* [MINOR][SQL] Fix comments styl and correct several styles and nits in CSV ↵hyukjinkwon2016-04-014-49/+48
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | data source ## What changes were proposed in this pull request? While trying to create a PR (which was not an issue at the end), I just corrected some style nits. So, I removed the changes except for some coding style corrections. - According to the [scala-style-guide#documentation-style](https://github.com/databricks/scala-style-guide#documentation-style), Scala style comments are discouraged. >```scala >/** This is a correct one-liner, short description. */ > >/** > * This is correct multi-line JavaDoc comment. And > * this is my second line, and if I keep typing, this would be > * my third line. > */ > >/** In Spark, we don't use the ScalaDoc style so this > * is not correct. > */ >``` - Double newlines between consecutive methods was removed. According to [scala-style-guide#blank-lines-vertical-whitespace](https://github.com/databricks/scala-style-guide#blank-lines-vertical-whitespace), single newline appears when >Between consecutive members (or initializers) of a class: fields, constructors, methods, nested classes, static initializers, instance initializers. - Remove uesless parentheses in tests - Use `mapPartitions` instead of `mapPartitionsWithIndex()`. ## How was this patch tested? Unit tests were used and `dev/run_tests` for style tests. Author: hyukjinkwon <gurwls223@gmail.com> Closes #12109 from HyukjinKwon/SPARK-14271.
* [SPARK-14285][SQL] Implement common type-safe aggregate functionsReynold Xin2016-04-019-111/+342
| | | | | | | | | | | | ## What changes were proposed in this pull request? In the Dataset API, it is fairly difficult for users to perform simple aggregations in a type-safe way at the moment because there are no aggregators that have been implemented. This pull request adds a few common aggregate functions in expressions.scala.typed package, and also creates the expressions.java.typed package without implementation. The java implementation should probably come as a separate pull request. One challenge there is to resolve the type difference between Scala primitive types and Java boxed types. ## How was this patch tested? Added unit tests for them. Author: Reynold Xin <rxin@databricks.com> Closes #12077 from rxin/SPARK-14285.
* [SPARK-14251][SQL] Add SQL command for printing out generated code for debuggingDongjoon Hyun2016-04-014-28/+35
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR implements `EXPLAIN CODEGEN` SQL command which returns generated codes like `debugCodegen`. In `spark-shell`, we don't need to `import debug` module. In `spark-sql`, we can use this SQL command now. **Before** ``` scala> import org.apache.spark.sql.execution.debug._ scala> sql("select 'a' as a group by 1").debugCodegen() Found 2 WholeStageCodegen subtrees. == Subtree 1 / 2 == ... Generated code: ... == Subtree 2 / 2 == ... Generated code: ... ``` **After** ``` scala> sql("explain extended codegen select 'a' as a group by 1").collect().foreach(println) [Found 2 WholeStageCodegen subtrees.] [== Subtree 1 / 2 ==] ... [] [Generated code:] ... [] [== Subtree 2 / 2 ==] ... [] [Generated code:] ... ``` ## How was this patch tested? Pass the Jenkins tests (including new testcases) Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12099 from dongjoon-hyun/SPARK-14251.
* [SPARK-14138] [SQL] [MASTER] Fix generated SpecificColumnarIterator code can ↵Kazuaki Ishizaki2016-04-012-5/+51
| | | | | | | | | | | | | | | | exceed JVM size limit for cached DataFrames ## What changes were proposed in this pull request? This PR reduces Java byte code size of method in ```SpecificColumnarIterator``` by using a approach to make a group for lot of ```ColumnAccessor``` instantiations or method calls (more than 200) into a method ## How was this patch tested? Added a new unit test, which includes large instantiations and method calls, to ```InMemoryColumnarQuerySuite``` Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #12108 from kiszk/SPARK-14138-master.
* [SPARK-14244][SQL] Don't use SizeBasedWindowFunction.n created on executor ↵Cheng Lian2016-04-011-6/+16
| | | | | | | | | | | | | | | | side when evaluating window functions ## What changes were proposed in this pull request? `SizeBasedWindowFunction.n` is a global singleton attribute created for evaluating size based aggregate window functions like `CUME_DIST`. However, this attribute gets different expression IDs when created on both driver side and executor side. This PR adds `withPartitionSize` method to `SizeBasedWindowFunction` so that we can easily rewrite `SizeBasedWindowFunction.n` on executor side. ## How was this patch tested? A test case is added in `HiveSparkSubmitSuite`, which supports launching multi-process clusters. Author: Cheng Lian <lian@databricks.com> Closes #12040 from liancheng/spark-14244-fix-sized-window-function.
* [SPARK-14255][SQL] Streaming AggregationMichael Armbrust2016-04-0124-289/+691
| | | | | | | | | | | | | | | | | | | | | | This PR adds the ability to perform aggregations inside of a `ContinuousQuery`. In order to implement this feature, the planning of aggregation has augmented with a new `StatefulAggregationStrategy`. Unlike batch aggregation, stateful-aggregation uses the `StateStore` (introduced in #11645) to persist the results of partial aggregation across different invocations. The resulting physical plan performs the aggregation using the following progression: - Partial Aggregation - Shuffle - Partial Merge (now there is at most 1 tuple per group) - StateStoreRestore (now there is 1 tuple from this batch + optionally one from the previous) - Partial Merge (now there is at most 1 tuple per group) - StateStoreSave (saves the tuple for the next batch) - Complete (output the current result of the aggregation) The following refactoring was also performed to allow us to plug into existing code: - The get/put implementation is taken from #12013 - The logic for breaking down and de-duping the physical execution of aggregation has been move into a new pattern `PhysicalAggregation` - The `AttributeReference` used to identify the result of an `AggregateFunction` as been moved into the `AggregateExpression` container. This change moves the reference into the same object as the other intermediate references used in aggregation and eliminates the need to pass around a `Map[(AggregateFunction, Boolean), Attribute]`. Further clean up (using a different aggregation container for logical/physical plans) is deferred to a followup. - Some planning logic is moved from the `SessionState` into the `QueryExecution` to make it easier to override in the streaming case. - The ability to write a `StreamTest` that checks only the output of the last batch has been added to simulate the future addition of output modes. Author: Michael Armbrust <michael@databricks.com> Closes #12048 from marmbrus/statefulAgg.
* [SPARK-14316][SQL] StateStoreCoordinator should extend ThreadSafeRpcEndpointShixiong Zhu2016-04-012-7/+5
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? RpcEndpoint is not thread safe and allows multiple messages to be processed at the same time. StateStoreCoordinator should use ThreadSafeRpcEndpoint. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #12100 from zsxwing/fix-StateStoreCoordinator.
* [SPARK-13674] [SQL] Add wholestage codegen support to SampleLiang-Chi Hsieh2016-04-014-14/+99
| | | | | | | | | | | | | | | | | JIRA: https://issues.apache.org/jira/browse/SPARK-13674 ## What changes were proposed in this pull request? Sample operator doesn't support wholestage codegen now. This pr is to add support to it. ## How was this patch tested? A test is added into `BenchmarkWholeStageCodegen`. Besides, all tests should be passed. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #11517 from viirya/add-wholestage-sample.
* [SPARK-14160] Time Windowing functions for DatasetsBurak Yavuz2016-04-012-0/+379
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds the function `window` as a column expression. `window` can be used to bucket rows into time windows given a time column. With this expression, performing time series analysis on batch data, as well as streaming data should become much more simpler. ### Usage Assume the following schema: `sensor_id, measurement, timestamp` To average 5 minute data every 1 minute (window length of 5 minutes, slide duration of 1 minute), we will use: ```scala df.groupBy(window("timestamp", “5 minutes”, “1 minute”), "sensor_id") .agg(mean("measurement").as("avg_meas")) ``` This will generate windows such as: ``` 09:00:00-09:05:00 09:01:00-09:06:00 09:02:00-09:07:00 ... ``` Intervals will start at every `slideDuration` starting at the unix epoch (1970-01-01 00:00:00 UTC). To start intervals at a different point of time, e.g. 30 seconds after a minute, the `startTime` parameter can be used. ```scala df.groupBy(window("timestamp", “5 minutes”, “1 minute”, "30 second"), "sensor_id") .agg(mean("measurement").as("avg_meas")) ``` This will generate windows such as: ``` 09:00:30-09:05:30 09:01:30-09:06:30 09:02:30-09:07:30 ... ``` Support for Python will be made in a follow up PR after this. ## How was this patch tested? This patch has some basic unit tests for the `TimeWindow` expression testing that the parameters pass validation, and it also has some unit/integration tests testing the correctness of the windowing and usability in complex operations (multi-column grouping, multi-column projections, joins). Author: Burak Yavuz <brkyvz@gmail.com> Author: Michael Armbrust <michael@databricks.com> Closes #12008 from brkyvz/df-time-window.
* [MINOR] [SQL] Update usage of `debug` by removing `typeCheck` and adding ↵Dongjoon Hyun2016-04-011-2/+2
| | | | | | | | | | | | | | | | | | `debugCodegen` ## What changes were proposed in this pull request? This PR updates the usage comments of `debug` according to the following commits. - [SPARK-9754](https://issues.apache.org/jira/browse/SPARK-9754) removed `typeCheck`. - [SPARK-14227](https://issues.apache.org/jira/browse/SPARK-14227) added `debugCodegen`. ## How was this patch tested? Manual. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12094 from dongjoon-hyun/minor_fix_debug_usage.
* [SPARK-14184][SQL] Support native execution of SHOW DATABASE command and fix ↵Dilip Biswal2016-04-015-15/+136
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | SHOW TABLE to use table identifier pattern ## What changes were proposed in this pull request? This PR addresses the following 1. Supports native execution of SHOW DATABASES command 2. Fixes SHOW TABLES to apply the identifier_with_wildcards pattern if supplied. SHOW TABLE syntax ``` SHOW TABLES [IN database_name] ['identifier_with_wildcards']; ``` SHOW DATABASES syntax ``` SHOW (DATABASES|SCHEMAS) [LIKE 'identifier_with_wildcards']; ``` ## How was this patch tested? Tests added in SQLQuerySuite (both hive and sql contexts) and DDLCommandSuite Note: Since the table name pattern was not working , tests are added in both SQLQuerySuite to verify the application of the table pattern. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #11991 from dilipbiswal/dkb_show_database.
* [SPARK-14295][SPARK-14274][SQL] Implements buildReader() for LibSVMCheng Lian2016-03-313-1/+14
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR implements `FileFormat.buildReader()` for the LibSVM data source. Besides that, a new interface method `prepareRead()` is added to `FileFormat`: ```scala def prepareRead( sqlContext: SQLContext, options: Map[String, String], files: Seq[FileStatus]): Map[String, String] = options ``` After migrating from `buildInternalScan()` to `buildReader()`, we lost the opportunity to collect necessary global information, since `buildReader()` works in a per-partition manner. For example, LibSVM needs to infer the total number of features if the `numFeatures` data source option is not set. Any necessary collected global information should be returned using the data source options map. By default, this method just returns the original options untouched. An alternative approach is to absorb `inferSchema()` into `prepareRead()`, since schema inference is also some kind of global information gathering. However, this approach wasn't chosen because schema inference is optional, while `prepareRead()` must be called whenever a `HadoopFsRelation` based data source relation is instantiated. One unaddressed problem is that, when `numFeatures` is absent, now the input data will be scanned twice. The `buildInternalScan()` code path doesn't need to do this because it caches the raw parsed RDD in memory before computing the total number of features. However, with `FileScanRDD`, the raw parsed RDD is created in a different way (e.g. partitioning) from the final RDD. ## How was this patch tested? Tested using existing test suites. Author: Cheng Lian <lian@databricks.com> Closes #12088 from liancheng/spark-14295-libsvm-build-reader.
* [SPARK-14267] [SQL] [PYSPARK] execute multiple Python UDFs within single batchDavies Liu2016-03-314-67/+120
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR support multiple Python UDFs within single batch, also improve the performance. ```python >>> from pyspark.sql.types import IntegerType >>> sqlContext.registerFunction("double", lambda x: x * 2, IntegerType()) >>> sqlContext.registerFunction("add", lambda x, y: x + y, IntegerType()) >>> sqlContext.sql("SELECT double(add(1, 2)), add(double(2), 1)").explain(True) == Parsed Logical Plan == 'Project [unresolvedalias('double('add(1, 2)), None),unresolvedalias('add('double(2), 1), None)] +- OneRowRelation$ == Analyzed Logical Plan == double(add(1, 2)): int, add(double(2), 1): int Project [double(add(1, 2))#14,add(double(2), 1)#15] +- Project [double(add(1, 2))#14,add(double(2), 1)#15] +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15] +- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18] +- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17] +- OneRowRelation$ == Optimized Logical Plan == Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15] +- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18] +- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17] +- OneRowRelation$ == Physical Plan == WholeStageCodegen : +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15] : +- INPUT +- !BatchPythonEvaluation [add(pythonUDF1#17, 1)], [pythonUDF0#16,pythonUDF1#17,pythonUDF0#18] +- !BatchPythonEvaluation [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17] +- Scan OneRowRelation[] ``` ## How was this patch tested? Added new tests. Using the following script to benchmark 1, 2 and 3 udfs, ``` df = sqlContext.range(1, 1 << 23, 1, 4) double = F.udf(lambda x: x * 2, LongType()) print df.select(double(df.id)).count() print df.select(double(df.id), double(df.id + 1)).count() print df.select(double(df.id), double(df.id + 1), double(df.id + 2)).count() ``` Here is the results: N | Before | After | speed up ---- |------------ | -------------|------ 1 | 22 s | 7 s | 3.1X 2 | 38 s | 13 s | 2.9X 3 | 58 s | 16 s | 3.6X This benchmark ran locally with 4 CPUs. For 3 UDFs, it launched 12 Python before before this patch, 4 process after this patch. After this patch, it will use less memory for multiple UDFs than before (less buffering). Author: Davies Liu <davies@databricks.com> Closes #12057 from davies/multi_udfs.
* [SPARK-14304][SQL][TESTS] Fix tests that don't create temp files in the ↵Shixiong Zhu2016-03-316-22/+24
| | | | | | | | | | | | | | | | `java.io.tmpdir` folder ## What changes were proposed in this pull request? If I press `CTRL-C` when running these tests, the temp files will be left in `sql/core` folder and I need to delete them manually. It's annoying. This PR just moves the temp files to the `java.io.tmpdir` folder and add a name prefix for them. ## How was this patch tested? Existing Jenkins tests Author: Shixiong Zhu <shixiong@databricks.com> Closes #12093 from zsxwing/temp-file.
* [SPARK-14182][SQL] Parse DDL Command: Alter Viewgatorsmile2016-03-313-43/+169
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This PR is to provide native parsing support for DDL commands: `Alter View`. Since its AST trees are highly similar to `Alter Table`. Thus, both implementation are integrated into the same one. Based on the Hive DDL document: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL and https://cwiki.apache.org/confluence/display/Hive/PartitionedViews **Syntax:** ```SQL ALTER VIEW view_name RENAME TO new_view_name ``` - to change the name of a view to a different name **Syntax:** ```SQL ALTER VIEW view_name SET TBLPROPERTIES ('comment' = new_comment); ``` - to add metadata to a view **Syntax:** ```SQL ALTER VIEW view_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key') ``` - to remove metadata from a view **Syntax:** ```SQL ALTER VIEW view_name ADD [IF NOT EXISTS] PARTITION spec1[, PARTITION spec2, ...] ``` - to add the partitioning metadata for a view. - the syntax of partition spec in `ALTER VIEW` is identical to `ALTER TABLE`, **EXCEPT** that it is **ILLEGAL** to specify a `LOCATION` clause. **Syntax:** ```SQL ALTER VIEW view_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] ``` - to drop the related partition metadata for a view. Added the related test cases to `DDLCommandSuite` Author: gatorsmile <gatorsmile@gmail.com> Author: xiaoli <lixiao1983@gmail.com> Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local> Closes #11987 from gatorsmile/parseAlterView.
* [SPARK-14278][SQL] Initialize columnar batch with proper memory modeSameer Agarwal2016-03-311-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fixes a minor bug in the record reader constructor that was possibly introduced during refactoring. ## How was this patch tested? N/A Author: Sameer Agarwal <sameer@databricks.com> Closes #12070 from sameeragarwal/vectorized-rr.
* [SPARK-14263][SQL] Benchmark Vectorized HashMap for GroupBy AggregatesSameer Agarwal2016-03-312-10/+142
| | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes a new data-structure based on a vectorized hashmap that can be potentially _codegened_ in `TungstenAggregate` to speed up aggregates with group by. Micro-benchmarks show a 10x improvement over the current `BytesToBytes` aggregation map. ## How was this patch tested? Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz BytesToBytesMap: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------- hash 108 / 119 96.9 10.3 1.0X fast hash 63 / 70 166.2 6.0 1.7X arrayEqual 70 / 73 150.8 6.6 1.6X Java HashMap (Long) 141 / 200 74.3 13.5 0.8X Java HashMap (two ints) 145 / 185 72.3 13.8 0.7X Java HashMap (UnsafeRow) 499 / 524 21.0 47.6 0.2X BytesToBytesMap (off Heap) 483 / 548 21.7 46.0 0.2X BytesToBytesMap (on Heap) 485 / 562 21.6 46.2 0.2X Vectorized Hashmap 54 / 60 193.7 5.2 2.0X Author: Sameer Agarwal <sameer@databricks.com> Closes #12055 from sameeragarwal/vectorized-hashmap.
* [SPARK-14211][SQL] Remove ANTLR3 based parserHerman van Hovell2016-03-315-841/+16
| | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? This PR removes the ANTLR3 based parser, and moves the new ANTLR4 based parser into the `org.apache.spark.sql.catalyst.parser package`. ### How was this patch tested? Existing unit tests. cc rxin andrewor14 yhuai Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #12071 from hvanhovell/SPARK-14211.
* [SPARK-14206][SQL] buildReader() implementation for CSVCheng Lian2016-03-308-56/+111
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Major changes: 1. Implement `FileFormat.buildReader()` for the CSV data source. 1. Add an extra argument to `FileFormat.buildReader()`, `physicalSchema`, which is basically the result of `FileFormat.inferSchema` or user specified schema. This argument is necessary because the CSV data source needs to know all the columns of the underlying files to read the file. ## How was this patch tested? Existing tests should do the work. Author: Cheng Lian <lian@databricks.com> Closes #12002 from liancheng/spark-14206-csv-build-reader.
* [SPARK-14081][SQL] - Preserve DataFrame column types when filling nulls.Travis Crawford2016-03-302-28/+40
| | | | | | | | | | | | ## What changes were proposed in this pull request? This change resolves an issue where `DataFrameNaFunctions.fill` changes a `FloatType` column to a `DoubleType`. We also clarify the contract that replacement values will be cast to the column data type, which may change the replacement value when casting to a lower precision type. ## How was this patch tested? This patch has associated unit tests. Author: Travis Crawford <travis@medium.com> Closes #11967 from traviscrawford/SPARK-14081-dataframena.
* [SPARK-14282][SQL] CodeFormatter should handle oneline comment with /* */ ↵Dongjoon Hyun2016-03-301-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | properly ## What changes were proposed in this pull request? This PR improves `CodeFormatter` to fix the following malformed indentations. ```java /* 019 */ public java.lang.Object apply(java.lang.Object _i) { /* 020 */ InternalRow i = (InternalRow) _i; /* 021 */ /* createexternalrow(if (isnull(input[0, double])) null else input[0, double], if (isnull(input[1, int])) null else input[1, int], ... */ /* 022 */ boolean isNull = false; /* 023 */ final Object[] values = new Object[2]; /* 024 */ /* if (isnull(input[0, double])) null else input[0, double] */ /* 025 */ /* isnull(input[0, double]) */ ... /* 053 */ if (!false && false) { /* 054 */ /* null */ /* 055 */ final int value9 = -1; /* 056 */ isNull6 = true; /* 057 */ value6 = value9; /* 058 */ } else { ... /* 077 */ return mutableRow; /* 078 */ } /* 079 */ } /* 080 */ ``` After this PR, the code will be formatted like the following. ```java /* 019 */ public java.lang.Object apply(java.lang.Object _i) { /* 020 */ InternalRow i = (InternalRow) _i; /* 021 */ /* createexternalrow(if (isnull(input[0, double])) null else input[0, double], if (isnull(input[1, int])) null else input[1, int], ... */ /* 022 */ boolean isNull = false; /* 023 */ final Object[] values = new Object[2]; /* 024 */ /* if (isnull(input[0, double])) null else input[0, double] */ /* 025 */ /* isnull(input[0, double]) */ ... /* 053 */ if (!false && false) { /* 054 */ /* null */ /* 055 */ final int value9 = -1; /* 056 */ isNull6 = true; /* 057 */ value6 = value9; /* 058 */ } else { ... /* 077 */ return mutableRow; /* 078 */ } /* 079 */ } /* 080 */ ``` Also, this issue fixes the following too. (Similar with [SPARK-14185](https://issues.apache.org/jira/browse/SPARK-14185)) ```java 16/03/30 12:39:24 DEBUG WholeStageCodegen: /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIterator(references); /* 003 */ } ``` ```java 16/03/30 12:46:32 DEBUG WholeStageCodegen: /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIterator(references); /* 003 */ } ``` ## How was this patch tested? Pass the Jenkins tests (including new CodeFormatterSuite testcases.) Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12072 from dongjoon-hyun/SPARK-14282.
* [SPARK-14259][SQL] Add a FileSourceStrategy option for limiting #files in a ↵Takeshi YAMAMURO2016-03-303-2/+59
| | | | | | | | | | | | | | partition ## What changes were proposed in this pull request? This pr is to add a config to control the maximum number of files as even small files have a non-trivial fixed cost. The current packing can put a lot of small files together which cases straggler tasks. ## How was this patch tested? I added tests to check if many files get split into partitions in FileSourceStrategySuite. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #12068 from maropu/SPARK-14259.
* [SPARK-14268][SQL] rename toRowExpressions and fromRowExpression to ↵Wenchen Fan2016-03-301-2/+2
| | | | | | | | | | | | | | | | serializer and deserializer in ExpressionEncoder ## What changes were proposed in this pull request? In `ExpressionEncoder`, we use `constructorFor` to build `fromRowExpression` as the `deserializer` in `ObjectOperator`. It's kind of confusing, we should make the name consistent. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #12058 from cloud-fan/rename.
* [SPARK-14114][SQL] implement buildReader for text data sourceWenchen Fan2016-03-302-2/+29
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR implements buildReader for text data source and enable it in the new data source code path. ## How was this patch tested? Existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #11934 from cloud-fan/text.
* [SPARK-14124][SQL] Implement Database-related DDL Commandsgatorsmile2016-03-295-59/+286
| | | | | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? This PR is to implement the following four Database-related DDL commands: - `CREATE DATABASE|SCHEMA [IF NOT EXISTS] database_name` - `DROP DATABASE [IF EXISTS] database_name [RESTRICT|CASCADE]` - `DESCRIBE DATABASE [EXTENDED] db_name` - `ALTER (DATABASE|SCHEMA) database_name SET DBPROPERTIES (property_name=property_value, ...)` Another PR will be submitted to handle the unsupported commands. In the Database-related DDL commands, we will issue an error exception for `ALTER (DATABASE|SCHEMA) database_name SET OWNER [USER|ROLE] user_or_role`. cc yhuai andrewor14 rxin Could you review the changes? Is it in the right direction? Thanks! #### How was this patch tested? Added a few test cases in `command/DDLSuite.scala` for testing DDL command execution in `SQLContext`. Since `HiveContext` also shares the same implementation, the existing test cases in `\hive` also verifies the correctness of these commands. Author: gatorsmile <gatorsmile@gmail.com> Author: xiaoli <lixiao1983@gmail.com> Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local> Closes #12009 from gatorsmile/dbDDL.
* [SPARK-14215] [SQL] [PYSPARK] Support chained Python UDFsDavies Liu2016-03-292-10/+45
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR brings the support for chained Python UDFs, for example ```sql select udf1(udf2(a)) select udf1(udf2(a) + 3) select udf1(udf2(a) + udf3(b)) ``` Also directly chained unary Python UDFs are put in single batch of Python UDFs, others may require multiple batches. For example, ```python >>> sqlContext.sql("select double(double(1))").explain() == Physical Plan == WholeStageCodegen : +- Project [pythonUDF#10 AS double(double(1))#9] : +- INPUT +- !BatchPythonEvaluation double(double(1)), [pythonUDF#10] +- Scan OneRowRelation[] >>> sqlContext.sql("select double(double(1) + double(2))").explain() == Physical Plan == WholeStageCodegen : +- Project [pythonUDF#19 AS double((double(1) + double(2)))#16] : +- INPUT +- !BatchPythonEvaluation double((pythonUDF#17 + pythonUDF#18)), [pythonUDF#17,pythonUDF#18,pythonUDF#19] +- !BatchPythonEvaluation double(2), [pythonUDF#17,pythonUDF#18] +- !BatchPythonEvaluation double(1), [pythonUDF#17] +- Scan OneRowRelation[] ``` TODO: will support multiple unrelated Python UDFs in one batch (another PR). ## How was this patch tested? Added new unit tests for chained UDFs. Author: Davies Liu <davies@databricks.com> Closes #12014 from davies/py_udfs.
* [SPARK-14227][SQL] Add method for printing out generated code for debuggingEric Liang2016-03-293-6/+60
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This adds `debugCodegen` to the debug package for query execution. ## How was this patch tested? Unit and manual testing. Output example: ``` scala> import org.apache.spark.sql.execution.debug._ import org.apache.spark.sql.execution.debug._ scala> sqlContext.range(100).groupBy("id").count().orderBy("id").debugCodegen() Found 3 WholeStageCodegen subtrees. == Subtree 1 / 3 == WholeStageCodegen : +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L]) : +- Range 0, 1, 1, 100, [id#0L] Generated code: /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIterator(references); /* 003 */ } /* 004 */ /* 005 */ /** Codegened pipeline for: /* 006 */ * TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L]) /* 007 */ +- Range 0, 1, 1, 100, [id#0L] /* 008 */ */ /* 009 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { /* 010 */ private Object[] references; /* 011 */ private boolean agg_initAgg; /* 012 */ private org.apache.spark.sql.execution.aggregate.TungstenAggregate agg_plan; /* 013 */ private org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap agg_hashMap; /* 014 */ private org.apache.spark.sql.execution.UnsafeKVExternalSorter agg_sorter; /* 015 */ private org.apache.spark.unsafe.KVIterator agg_mapIter; /* 016 */ private org.apache.spark.sql.execution.metric.LongSQLMetric range_numOutputRows; /* 017 */ private org.apache.spark.sql.execution.metric.LongSQLMetricValue range_metricValue; /* 018 */ private boolean range_initRange; /* 019 */ private long range_partitionEnd; /* 020 */ private long range_number; /* 021 */ private boolean range_overflow; /* 022 */ private scala.collection.Iterator range_input; /* 023 */ private UnsafeRow range_result; /* 024 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder range_holder; /* 025 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter range_rowWriter; /* 026 */ private UnsafeRow agg_result; /* 027 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder; /* 028 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter; /* 029 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowJoiner agg_unsafeRowJoiner; /* 030 */ private org.apache.spark.sql.execution.metric.LongSQLMetric wholestagecodegen_numOutputRows; /* 031 */ private org.apache.spark.sql.execution.metric.LongSQLMetricValue wholestagecodegen_metricValue; /* 032 */ /* 033 */ public GeneratedIterator(Object[] references) { /* 034 */ this.references = references; /* 035 */ } /* 036 */ /* 037 */ public void init(scala.collection.Iterator inputs[]) { /* 038 */ agg_initAgg = false; /* 039 */ this.agg_plan = (org.apache.spark.sql.execution.aggregate.TungstenAggregate) references[0]; /* 040 */ agg_hashMap = agg_plan.createHashMap(); /* 041 */ /* 042 */ this.range_numOutputRows = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[1]; /* 043 */ range_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) range_numOutputRows.localValue(); /* 044 */ range_initRange = false; /* 045 */ range_partitionEnd = 0L; /* 046 */ range_number = 0L; /* 047 */ range_overflow = false; /* 048 */ range_input = inputs[0]; /* 049 */ range_result = new UnsafeRow(1); /* 050 */ this.range_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(range_result, 0); /* 051 */ this.range_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(range_holder, 1); /* 052 */ agg_result = new UnsafeRow(1); /* 053 */ this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0); /* 054 */ this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 1); /* 055 */ agg_unsafeRowJoiner = agg_plan.createUnsafeJoiner(); /* 056 */ this.wholestagecodegen_numOutputRows = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[2]; /* 057 */ wholestagecodegen_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) wholestagecodegen_numOutputRows.localValue(); /* 058 */ } /* 059 */ /* 060 */ private void agg_doAggregateWithKeys() throws java.io.IOException { /* 061 */ /*** PRODUCE: Range 0, 1, 1, 100, [id#0L] */ /* 062 */ /* 063 */ // initialize Range /* 064 */ if (!range_initRange) { /* 065 */ range_initRange = true; /* 066 */ if (range_input.hasNext()) { /* 067 */ initRange(((InternalRow) range_input.next()).getInt(0)); /* 068 */ } else { /* 069 */ return; /* 070 */ } /* 071 */ } /* 072 */ /* 073 */ while (!range_overflow && range_number < range_partitionEnd) { /* 074 */ long range_value = range_number; /* 075 */ range_number += 1L; /* 076 */ if (range_number < range_value ^ 1L < 0) { /* 077 */ range_overflow = true; /* 078 */ } /* 079 */ /* 080 */ /*** CONSUME: TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L]) */ /* 081 */ /* 082 */ // generate grouping key /* 083 */ agg_rowWriter.write(0, range_value); /* 084 */ /* hash(input[0, bigint], 42) */ /* 085 */ int agg_value1 = 42; /* 086 */ /* 087 */ agg_value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashLong(range_value, agg_value1); /* 088 */ UnsafeRow agg_aggBuffer = null; /* 089 */ if (true) { /* 090 */ // try to get the buffer from hash map /* 091 */ agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value1); /* 092 */ } /* 093 */ if (agg_aggBuffer == null) { /* 094 */ if (agg_sorter == null) { /* 095 */ agg_sorter = agg_hashMap.destructAndCreateExternalSorter(); /* 096 */ } else { /* 097 */ agg_sorter.merge(agg_hashMap.destructAndCreateExternalSorter()); /* 098 */ } /* 099 */ /* 100 */ // the hash map had be spilled, it should have enough memory now, /* 101 */ // try to allocate buffer again. /* 102 */ agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value1); /* 103 */ if (agg_aggBuffer == null) { /* 104 */ // failed to allocate the first page /* 105 */ throw new OutOfMemoryError("No enough memory for aggregation"); /* 106 */ } /* 107 */ } /* 108 */ /* 109 */ // evaluate aggregate function /* 110 */ /* (input[0, bigint] + 1) */ /* 111 */ /* input[0, bigint] */ /* 112 */ long agg_value4 = agg_aggBuffer.getLong(0); /* 113 */ /* 114 */ long agg_value3 = -1L; /* 115 */ agg_value3 = agg_value4 + 1L; /* 116 */ // update aggregate buffer /* 117 */ agg_aggBuffer.setLong(0, agg_value3); /* 118 */ /* 119 */ if (shouldStop()) return; /* 120 */ } /* 121 */ /* 122 */ agg_mapIter = agg_plan.finishAggregate(agg_hashMap, agg_sorter); /* 123 */ } /* 124 */ /* 125 */ private void initRange(int idx) { /* 126 */ java.math.BigInteger index = java.math.BigInteger.valueOf(idx); /* 127 */ java.math.BigInteger numSlice = java.math.BigInteger.valueOf(1L); /* 128 */ java.math.BigInteger numElement = java.math.BigInteger.valueOf(100L); /* 129 */ java.math.BigInteger step = java.math.BigInteger.valueOf(1L); /* 130 */ java.math.BigInteger start = java.math.BigInteger.valueOf(0L); /* 131 */ /* 132 */ java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start); /* 133 */ if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) { /* 134 */ range_number = Long.MAX_VALUE; /* 135 */ } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) { /* 136 */ range_number = Long.MIN_VALUE; /* 137 */ } else { /* 138 */ range_number = st.longValue(); /* 139 */ } /* 140 */ /* 141 */ java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice) /* 142 */ .multiply(step).add(start); /* 143 */ if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) { /* 144 */ range_partitionEnd = Long.MAX_VALUE; /* 145 */ } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) { /* 146 */ range_partitionEnd = Long.MIN_VALUE; /* 147 */ } else { /* 148 */ range_partitionEnd = end.longValue(); /* 149 */ } /* 150 */ /* 151 */ range_metricValue.add((range_partitionEnd - range_number) / 1L); /* 152 */ } /* 153 */ /* 154 */ protected void processNext() throws java.io.IOException { /* 155 */ /*** PRODUCE: TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L]) */ /* 156 */ /* 157 */ if (!agg_initAgg) { /* 158 */ agg_initAgg = true; /* 159 */ agg_doAggregateWithKeys(); /* 160 */ } /* 161 */ /* 162 */ // output the result /* 163 */ while (agg_mapIter.next()) { /* 164 */ wholestagecodegen_metricValue.add(1); /* 165 */ UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey(); /* 166 */ UnsafeRow agg_aggBuffer1 = (UnsafeRow) agg_mapIter.getValue(); /* 167 */ /* 168 */ UnsafeRow agg_resultRow = agg_unsafeRowJoiner.join(agg_aggKey, agg_aggBuffer1); /* 169 */ /* 170 */ /*** CONSUME: WholeStageCodegen */ /* 171 */ /* 172 */ append(agg_resultRow); /* 173 */ /* 174 */ if (shouldStop()) return; /* 175 */ } /* 176 */ /* 177 */ agg_mapIter.close(); /* 178 */ if (agg_sorter == null) { /* 179 */ agg_hashMap.free(); /* 180 */ } /* 181 */ } /* 182 */ } == Subtree 2 / 3 == WholeStageCodegen : +- Sort [id#0L ASC], true, 0 : +- INPUT +- Exchange rangepartitioning(id#0L ASC, 200), None +- WholeStageCodegen : +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L]) : +- INPUT +- Exchange hashpartitioning(id#0L, 200), None +- WholeStageCodegen : +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L]) : +- Range 0, 1, 1, 100, [id#0L] Generated code: /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIterator(references); /* 003 */ } /* 004 */ /* 005 */ /** Codegened pipeline for: /* 006 */ * Sort [id#0L ASC], true, 0 /* 007 */ +- INPUT /* 008 */ */ /* 009 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { /* 010 */ private Object[] references; /* 011 */ private boolean sort_needToSort; /* 012 */ private org.apache.spark.sql.execution.Sort sort_plan; /* 013 */ private org.apache.spark.sql.execution.UnsafeExternalRowSorter sort_sorter; /* 014 */ private org.apache.spark.executor.TaskMetrics sort_metrics; /* 015 */ private scala.collection.Iterator<UnsafeRow> sort_sortedIter; /* 016 */ private scala.collection.Iterator inputadapter_input; /* 017 */ private org.apache.spark.sql.execution.metric.LongSQLMetric sort_dataSize; /* 018 */ private org.apache.spark.sql.execution.metric.LongSQLMetricValue sort_metricValue; /* 019 */ private org.apache.spark.sql.execution.metric.LongSQLMetric sort_spillSize; /* 020 */ private org.apache.spark.sql.execution.metric.LongSQLMetricValue sort_metricValue1; /* 021 */ /* 022 */ public GeneratedIterator(Object[] references) { /* 023 */ this.references = references; /* 024 */ } /* 025 */ /* 026 */ public void init(scala.collection.Iterator inputs[]) { /* 027 */ sort_needToSort = true; /* 028 */ this.sort_plan = (org.apache.spark.sql.execution.Sort) references[0]; /* 029 */ sort_sorter = sort_plan.createSorter(); /* 030 */ sort_metrics = org.apache.spark.TaskContext.get().taskMetrics(); /* 031 */ /* 032 */ inputadapter_input = inputs[0]; /* 033 */ this.sort_dataSize = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[1]; /* 034 */ sort_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) sort_dataSize.localValue(); /* 035 */ this.sort_spillSize = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[2]; /* 036 */ sort_metricValue1 = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) sort_spillSize.localValue(); /* 037 */ } /* 038 */ /* 039 */ private void sort_addToSorter() throws java.io.IOException { /* 040 */ /*** PRODUCE: INPUT */ /* 041 */ /* 042 */ while (inputadapter_input.hasNext()) { /* 043 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 044 */ /*** CONSUME: Sort [id#0L ASC], true, 0 */ /* 045 */ /* 046 */ sort_sorter.insertRow((UnsafeRow)inputadapter_row); /* 047 */ if (shouldStop()) return; /* 048 */ } /* 049 */ /* 050 */ } /* 051 */ /* 052 */ protected void processNext() throws java.io.IOException { /* 053 */ /*** PRODUCE: Sort [id#0L ASC], true, 0 */ /* 054 */ if (sort_needToSort) { /* 055 */ sort_addToSorter(); /* 056 */ Long sort_spillSizeBefore = sort_metrics.memoryBytesSpilled(); /* 057 */ sort_sortedIter = sort_sorter.sort(); /* 058 */ sort_metricValue.add(sort_sorter.getPeakMemoryUsage()); /* 059 */ sort_metricValue1.add(sort_metrics.memoryBytesSpilled() - sort_spillSizeBefore); /* 060 */ sort_metrics.incPeakExecutionMemory(sort_sorter.getPeakMemoryUsage()); /* 061 */ sort_needToSort = false; /* 062 */ } /* 063 */ /* 064 */ while (sort_sortedIter.hasNext()) { /* 065 */ UnsafeRow sort_outputRow = (UnsafeRow)sort_sortedIter.next(); /* 066 */ /* 067 */ /*** CONSUME: WholeStageCodegen */ /* 068 */ /* 069 */ append(sort_outputRow); /* 070 */ /* 071 */ if (shouldStop()) return; /* 072 */ } /* 073 */ } /* 074 */ } == Subtree 3 / 3 == WholeStageCodegen : +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L]) : +- INPUT +- Exchange hashpartitioning(id#0L, 200), None +- WholeStageCodegen : +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L]) : +- Range 0, 1, 1, 100, [id#0L] Generated code: /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIterator(references); /* 003 */ } /* 004 */ /* 005 */ /** Codegened pipeline for: /* 006 */ * TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L]) /* 007 */ +- INPUT /* 008 */ */ /* 009 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { /* 010 */ private Object[] references; /* 011 */ private boolean agg_initAgg; /* 012 */ private org.apache.spark.sql.execution.aggregate.TungstenAggregate agg_plan; /* 013 */ private org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap agg_hashMap; /* 014 */ private org.apache.spark.sql.execution.UnsafeKVExternalSorter agg_sorter; /* 015 */ private org.apache.spark.unsafe.KVIterator agg_mapIter; /* 016 */ private scala.collection.Iterator inputadapter_input; /* 017 */ private UnsafeRow agg_result; /* 018 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder; /* 019 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter; /* 020 */ private UnsafeRow agg_result1; /* 021 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder1; /* 022 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter1; /* 023 */ private org.apache.spark.sql.execution.metric.LongSQLMetric wholestagecodegen_numOutputRows; /* 024 */ private org.apache.spark.sql.execution.metric.LongSQLMetricValue wholestagecodegen_metricValue; /* 025 */ /* 026 */ public GeneratedIterator(Object[] references) { /* 027 */ this.references = references; /* 028 */ } /* 029 */ /* 030 */ public void init(scala.collection.Iterator inputs[]) { /* 031 */ agg_initAgg = false; /* 032 */ this.agg_plan = (org.apache.spark.sql.execution.aggregate.TungstenAggregate) references[0]; /* 033 */ agg_hashMap = agg_plan.createHashMap(); /* 034 */ /* 035 */ inputadapter_input = inputs[0]; /* 036 */ agg_result = new UnsafeRow(1); /* 037 */ this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0); /* 038 */ this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 1); /* 039 */ agg_result1 = new UnsafeRow(2); /* 040 */ this.agg_holder1 = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result1, 0); /* 041 */ this.agg_rowWriter1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder1, 2); /* 042 */ this.wholestagecodegen_numOutputRows = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[1]; /* 043 */ wholestagecodegen_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) wholestagecodegen_numOutputRows.localValue(); /* 044 */ } /* 045 */ /* 046 */ private void agg_doAggregateWithKeys() throws java.io.IOException { /* 047 */ /*** PRODUCE: INPUT */ /* 048 */ /* 049 */ while (inputadapter_input.hasNext()) { /* 050 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 051 */ /*** CONSUME: TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L]) */ /* 052 */ /* input[0, bigint] */ /* 053 */ long inputadapter_value = inputadapter_row.getLong(0); /* 054 */ /* input[1, bigint] */ /* 055 */ long inputadapter_value1 = inputadapter_row.getLong(1); /* 056 */ /* 057 */ // generate grouping key /* 058 */ agg_rowWriter.write(0, inputadapter_value); /* 059 */ /* hash(input[0, bigint], 42) */ /* 060 */ int agg_value1 = 42; /* 061 */ /* 062 */ agg_value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashLong(inputadapter_value, agg_value1); /* 063 */ UnsafeRow agg_aggBuffer = null; /* 064 */ if (true) { /* 065 */ // try to get the buffer from hash map /* 066 */ agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value1); /* 067 */ } /* 068 */ if (agg_aggBuffer == null) { /* 069 */ if (agg_sorter == null) { /* 070 */ agg_sorter = agg_hashMap.destructAndCreateExternalSorter(); /* 071 */ } else { /* 072 */ agg_sorter.merge(agg_hashMap.destructAndCreateExternalSorter()); /* 073 */ } /* 074 */ /* 075 */ // the hash map had be spilled, it should have enough memory now, /* 076 */ // try to allocate buffer again. /* 077 */ agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value1); /* 078 */ if (agg_aggBuffer == null) { /* 079 */ // failed to allocate the first page /* 080 */ throw new OutOfMemoryError("No enough memory for aggregation"); /* 081 */ } /* 082 */ } /* 083 */ /* 084 */ // evaluate aggregate function /* 085 */ /* (input[0, bigint] + input[2, bigint]) */ /* 086 */ /* input[0, bigint] */ /* 087 */ long agg_value4 = agg_aggBuffer.getLong(0); /* 088 */ /* 089 */ long agg_value3 = -1L; /* 090 */ agg_value3 = agg_value4 + inputadapter_value1; /* 091 */ // update aggregate buffer /* 092 */ agg_aggBuffer.setLong(0, agg_value3); /* 093 */ if (shouldStop()) return; /* 094 */ } /* 095 */ /* 096 */ agg_mapIter = agg_plan.finishAggregate(agg_hashMap, agg_sorter); /* 097 */ } /* 098 */ /* 099 */ protected void processNext() throws java.io.IOException { /* 100 */ /*** PRODUCE: TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L]) */ /* 101 */ /* 102 */ if (!agg_initAgg) { /* 103 */ agg_initAgg = true; /* 104 */ agg_doAggregateWithKeys(); /* 105 */ } /* 106 */ /* 107 */ // output the result /* 108 */ while (agg_mapIter.next()) { /* 109 */ wholestagecodegen_metricValue.add(1); /* 110 */ UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey(); /* 111 */ UnsafeRow agg_aggBuffer1 = (UnsafeRow) agg_mapIter.getValue(); /* 112 */ /* 113 */ /* input[0, bigint] */ /* 114 */ long agg_value6 = agg_aggKey.getLong(0); /* 115 */ /* input[0, bigint] */ /* 116 */ long agg_value7 = agg_aggBuffer1.getLong(0); /* 117 */ /* 118 */ /*** CONSUME: WholeStageCodegen */ /* 119 */ /* 120 */ agg_rowWriter1.write(0, agg_value6); /* 121 */ /* 122 */ agg_rowWriter1.write(1, agg_value7); /* 123 */ append(agg_result1); /* 124 */ /* 125 */ if (shouldStop()) return; /* 126 */ } /* 127 */ /* 128 */ agg_mapIter.close(); /* 129 */ if (agg_sorter == null) { /* 130 */ agg_hashMap.free(); /* 131 */ } /* 132 */ } /* 133 */ } ``` rxin Author: Eric Liang <ekl@databricks.com> Closes #12025 from ericl/spark-14227.
* [MINOR][SQL] Fix exception message to print string-array correctly.Dongjoon Hyun2016-03-291-1/+3
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR is a simple fix for an exception message to print `string[]` content correctly. ```java String[] colPath = requestedSchema.getPaths().get(i); ... - throw new IOException("Required column is missing in data file. Col: " + colPath); + throw new IOException("Required column is missing in data file. Col: " + Arrays.toString(colPath)); ``` ## How was this patch tested? Manual. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12041 from dongjoon-hyun/fix_exception_message_with_string_array.
* [SPARK-14208][SQL] Renames spark.sql.parquet.fileScanCheng Lian2016-03-292-5/+5
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Renames SQL option `spark.sql.parquet.fileScan` since now all `HadoopFsRelation` based data sources are being migrated to `FileScanRDD` code path. ## How was this patch tested? None. Author: Cheng Lian <lian@databricks.com> Closes #12003 from liancheng/spark-14208-option-renaming.
* [SPARK-14158][SQL] implement buildReader for json data sourceWenchen Fan2016-03-294-4/+90
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR implements buildReader for json data source and enable it in the new data source code path. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #11960 from cloud-fan/json.
* [SPARK-14210] [SQL] Add a metric for time spent in scans.Nong Li2016-03-281-63/+94
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This adds a metric to parquet scans that measures the time in just the scan phase. This is only possible when the scan returns ColumnarBatches, otherwise the overhead is too high. This combined with the pipeline metric lets us easily see what percent of the time was in the scan. Author: Nong Li <nong@databricks.com> Closes #12007 from nongli/spark-14210.
* [SPARK-13981][SQL] Defer evaluating variables within Filter operator.Nong Li2016-03-281-16/+61
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This improves the Filter codegen for NULLs by deferring loading the values for IsNotNull. Instead of generating code like: boolean isNull = ... int value = ... if (isNull) continue; we will generate: boolean isNull = ... if (isNull) continue; int value = ... This is useful since retrieving the values can be non-trivial (they can be dictionary encoded among other things). This currently only works when the attribute comes from the column batch but could be extended to other cases in the future. ## How was this patch tested? On tpcds q55, this fixes the regression from introducing the IsNotNull predicates. ``` TPCDS Snappy: Best/Avg Time(ms) Rate(M/s) Per Row(ns) -------------------------------------------------------------------------------- q55 4564 / 5036 25.2 39.6 q55 4064 / 4340 28.3 35.3 ``` Author: Nong Li <nong@databricks.com> Closes #11792 from nongli/spark-13981.
* [SPARK-14205][SQL] remove trait QueryableWenchen Fan2016-03-286-161/+95
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? After DataFrame and Dataset are merged, the trait `Queryable` becomes unnecessary as it has only one implementation. We should remove it. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #12001 from cloud-fan/df-ds.
* [SPARK-14013][SQL] Proper temp function support in catalogAndrew Or2016-03-281-4/+4
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Session catalog was added in #11750. However, it doesn't really support temporary functions properly; right now we only store the metadata in the form of `CatalogFunction`, but this doesn't make sense for temporary functions because there is no class name. This patch moves the `FunctionRegistry` into the `SessionCatalog`. With this, the user can call `catalog.createTempFunction` and `catalog.lookupFunction` to use the function they registered previously. This is currently still dead code, however. ## How was this patch tested? `SessionCatalogSuite`. Author: Andrew Or <andrew@databricks.com> Closes #11972 from andrewor14/temp-functions.
* [SPARK-14169][CORE] Add UninterruptibleThreadShixiong Zhu2016-03-281-66/+8
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Extract the workaround for HADOOP-10622 introduced by #11940 into UninterruptibleThread so that we can test and reuse it. ## How was this patch tested? Unit tests Author: Shixiong Zhu <shixiong@databricks.com> Closes #11971 from zsxwing/uninterrupt.