aboutsummaryrefslogtreecommitdiff
path: root/sql
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-16289][SQL] Implement posexplode table generating functionDongjoon Hyun2016-06-309-71/+232
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR implements `posexplode` table generating function. Currently, master branch raises the following exception for `map` argument. It's different from Hive. **Before** ```scala scala> sql("select posexplode(map('a', 1, 'b', 2))").show org.apache.spark.sql.AnalysisException: No handler for Hive UDF ... posexplode() takes an array as a parameter; line 1 pos 7 ``` **After** ```scala scala> sql("select posexplode(map('a', 1, 'b', 2))").show +---+---+-----+ |pos|key|value| +---+---+-----+ | 0| a| 1| | 1| b| 2| +---+---+-----+ ``` For `array` argument, `after` is the same with `before`. ``` scala> sql("select posexplode(array(1, 2, 3))").show +---+---+ |pos|col| +---+---+ | 0| 1| | 1| 2| | 2| 3| +---+---+ ``` ## How was this patch tested? Pass the Jenkins tests with newly added testcases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13971 from dongjoon-hyun/SPARK-16289.
* [SPARK-13850] Force the sorter to Spill when number of elements in th…Sital Kedia2016-06-309-6/+30
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Force the sorter to Spill when number of elements in the pointer array reach a certain size. This is to workaround the issue of timSort failing on large buffer size. ## How was this patch tested? Tested by running a job which was failing without this change due to TimSort bug. Author: Sital Kedia <skedia@fb.com> Closes #13107 from sitalkedia/fix_TimSort.
* [SPARK-15820][PYSPARK][SQL] Add Catalog.refreshTable into python APIWeichenXu2016-06-301-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add Catalog.refreshTable API into python interface for Spark-SQL. ## How was this patch tested? Existing test. Author: WeichenXu <WeichenXu123@outlook.com> Closes #13558 from WeichenXu123/update_python_sql_interface_refreshTable.
* [SPARK-16071][SQL] Checks size limit when doubling the array size in ↵Sean Zhong2016-06-302-2/+53
| | | | | | | | | | | | | | | | BufferHolder ## What changes were proposed in this pull request? This PR Checks the size limit when doubling the array size in BufferHolder to avoid integer overflow. ## How was this patch tested? Manual test. Author: Sean Zhong <seanzhong@databricks.com> Closes #13829 from clockfly/SPARK-16071_2.
* [SPARK-16274][SQL] Implement xpath_booleanpetermaxlee2016-06-305-1/+154
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch implements xpath_boolean expression for Spark SQL, a xpath function that returns true or false. The implementation is modelled after Hive's xpath_boolean, except that how the expression handles null inputs. Hive throws a NullPointerException at runtime if either of the input is null. This implementation returns null if either of the input is null. ## How was this patch tested? Created two new test suites. One for unit tests covering the expression, and the other for end-to-end test in SQL. Author: petermaxlee <petermaxlee@gmail.com> Closes #13964 from petermaxlee/SPARK-16274.
* [SPARK-16267][TEST] Replace deprecated `CREATE TEMPORARY TABLE ... USING` ↵Dongjoon Hyun2016-06-297-18/+18
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | from testsuites. ## What changes were proposed in this pull request? After SPARK-15674, `DDLStrategy` prints out the following deprecation messages in the testsuites. ``` 12:10:53.284 WARN org.apache.spark.sql.execution.SparkStrategies$DDLStrategy: CREATE TEMPORARY TABLE normal_orc_source USING... is deprecated, please use CREATE TEMPORARY VIEW viewName USING... instead ``` Total : 40 - JDBCWriteSuite: 14 - DDLSuite: 6 - TableScanSuite: 6 - ParquetSourceSuite: 5 - OrcSourceSuite: 2 - SQLQuerySuite: 2 - HiveCommandSuite: 2 - JsonSuite: 1 - PrunedScanSuite: 1 - FilteredScanSuite 1 This PR replaces `CREATE TEMPORARY TABLE` with `CREATE TEMPORARY VIEW` in order to remove the deprecation messages in the above testsuites except `DDLSuite`, `SQLQuerySuite`, `HiveCommandSuite`. The Jenkins results shows only remaining 10 messages. https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61422/consoleFull ## How was this patch tested? This is a testsuite-only change. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13956 from dongjoon-hyun/SPARK-16267.
* [SPARK-16134][SQL] optimizer rules for typed filterWenchen Fan2016-06-308-91/+162
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds 3 optimizer rules for typed filter: 1. push typed filter down through `SerializeFromObject` and eliminate the deserialization in filter condition. 2. pull typed filter up through `SerializeFromObject` and eliminate the deserialization in filter condition. 3. combine adjacent typed filters and share the deserialized object among all the condition expressions. This PR also adds `TypedFilter` logical plan, to separate it from normal filter, so that the concept is more clear and it's easier to write optimizer rules. ## How was this patch tested? `TypedFilterOptimizationSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #13846 from cloud-fan/filter.
* [SPARK-16228][SQL] HiveSessionCatalog should return `double`-param functions ↵Dongjoon Hyun2016-06-292-1/+22
| | | | | | | | | | | | | | | | | | | | | | | | | for decimal param lookups ## What changes were proposed in this pull request? This PR supports a fallback lookup by casting `DecimalType` into `DoubleType` for the external functions with `double`-type parameter. **Reported Error Scenarios** ```scala scala> sql("select percentile(value, 0.5) from values 1,2,3 T(value)") org.apache.spark.sql.AnalysisException: ... No matching method for class org.apache.hadoop.hive.ql.udf.UDAFPercentile with (int, decimal(38,18)). Possible choices: _FUNC_(bigint, array<double>) _FUNC_(bigint, double) ; line 1 pos 7 scala> sql("select percentile_approx(value, 0.5) from values 1.0,2.0,3.0 T(value)") org.apache.spark.sql.AnalysisException: ... Only a float/double or float/double array argument is accepted as parameter 2, but decimal(38,18) was passed instead.; line 1 pos 7 ``` ## How was this patch tested? Pass the Jenkins tests (including a new testcase). Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13930 from dongjoon-hyun/SPARK-16228.
* [SPARK-16238] Metrics for generated method and class bytecode sizeEric Liang2016-06-292-1/+43
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This extends SPARK-15860 to include metrics for the actual bytecode size of janino-generated methods. They can be accessed in the same way as any other codahale metric, e.g. ``` scala> org.apache.spark.metrics.source.CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.getSnapshot().getValues() res7: Array[Long] = Array(532, 532, 532, 542, 1479, 2670, 3585, 3585) scala> org.apache.spark.metrics.source.CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.getSnapshot().getValues() res8: Array[Long] = Array(5, 5, 5, 5, 10, 10, 10, 10, 15, 15, 15, 38, 63, 79, 88, 94, 94, 94, 132, 132, 165, 165, 220, 220) ``` ## How was this patch tested? Small unit test, also verified manually that the performance impact is minimal (<10%). hvanhovell Author: Eric Liang <ekl@databricks.com> Closes #13934 from ericl/spark-16238.
* [SPARK-16006][SQL] Attemping to write empty DataFrame with no fields throws ↵Dongjoon Hyun2016-06-292-2/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | non-intuitive exception ## What changes were proposed in this pull request? This PR allows `emptyDataFrame.write` since the user didn't specify any partition columns. **Before** ```scala scala> spark.emptyDataFrame.write.parquet("/tmp/t1") org.apache.spark.sql.AnalysisException: Cannot use all columns for partition columns; scala> spark.emptyDataFrame.write.csv("/tmp/t1") org.apache.spark.sql.AnalysisException: Cannot use all columns for partition columns; ``` After this PR, there occurs no exceptions and the created directory has only one file, `_SUCCESS`, as expected. ## How was this patch tested? Pass the Jenkins tests including updated test cases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13730 from dongjoon-hyun/SPARK-16006.
* [SPARK-16301] [SQL] The analyzer rule for resolving using joins should ↵Yin Huai2016-06-292-7/+49
| | | | | | | | | | | | | | respect the case sensitivity setting. ## What changes were proposed in this pull request? The analyzer rule for resolving using joins should respect the case sensitivity setting. ## How was this patch tested? New tests in ResolveNaturalJoinSuite Author: Yin Huai <yhuai@databricks.com> Closes #13977 from yhuai/SPARK-16301.
* [SPARK-14480][SQL] Remove meaningless StringIteratorReader for CSV data source.hyukjinkwon2016-06-294-291/+33
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR removes meaningless `StringIteratorReader` for CSV data source. In `CSVParser.scala`, there is an `Reader` wrapping `Iterator` but there are two problems by this. Firstly, it was actually not faster than processing line by line with Iterator due to additional logics to wrap `Iterator` to `Reader`. Secondly, this brought a bit of complexity because it needs additional logics to allow every line to be read bytes by bytes. So, it was pretty difficult to figure out issues about parsing, (eg. SPARK-14103). A benchmark was performed manually and the results were below: - Original codes with Reader wrapping Iterator |End-to-end (ns) | Parse Time (ns) | |-----------------------|------------------------| |14116265034 |2008277960 | - New codes with Iterator |End-to-end (ns) | Parse Time (ns) | |-----------------------|------------------------| |13451699644 | 1549050564 | For the details for the environment, dataset and methods, please refer the JIRA ticket. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #13808 from HyukjinKwon/SPARK-14480-small.
* [SPARK-16157][SQL] Add New Methods for comments in StructField and StructTypegatorsmile2016-06-2910-21/+125
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? Based on the previous discussion with cloud-fan hvanhovell in another related PR https://github.com/apache/spark/pull/13764#discussion_r67994276, it looks reasonable to add convenience methods for users to add `comment` when defining `StructField`. Currently, the column-related `comment` attribute is stored in `Metadata` of `StructField`. For example, users can add the `comment` attribute using the following way: ```Scala StructType( StructField( "cl1", IntegerType, nullable = false, new MetadataBuilder().putString("comment", "test").build()) :: Nil) ``` This PR is to add more user friendly methods for the `comment` attribute when defining a `StructField`. After the changes, users are provided three different ways to do it: ```Scala val struct = (new StructType) .add("a", "int", true, "test1") val struct = (new StructType) .add("c", StringType, true, "test3") val struct = (new StructType) .add(StructField("d", StringType).withComment("test4")) ``` #### How was this patch tested? Added test cases: - `DataTypeSuite` is for testing three types of API changes, - `DataFrameReaderWriterSuite` is for parquet, json and csv formats - using in-memory catalog - `OrcQuerySuite.scala` is for orc format using Hive-metastore Author: gatorsmile <gatorsmile@gmail.com> Closes #13860 from gatorsmile/newMethodForComment.
* [SPARK-16291][SQL] CheckAnalysis should capture nested aggregate functions ↵Cheng Lian2016-06-293-5/+12
| | | | | | | | | | | | | | | | | | | | that reference no input attributes ## What changes were proposed in this pull request? `MAX(COUNT(*))` is invalid since aggregate expression can't be nested within another aggregate expression. This case should be captured at analysis phase, but somehow sneaks off to runtime. The reason is that when checking aggregate expressions in `CheckAnalysis`, a checking branch treats all expressions that reference no input attributes as valid ones. However, `MAX(COUNT(*))` is translated into `MAX(COUNT(1))` at analysis phase and also references no input attribute. This PR fixes this issue by removing the aforementioned branch. ## How was this patch tested? New test case added in `AnalysisErrorSuite`. Author: Cheng Lian <lian@databricks.com> Closes #13968 from liancheng/spark-16291-nested-agg-functions.
* [TRIVIAL][DOCS][STREAMING][SQL] The return type mentioned in the Javadoc is ↵Holden Karau2016-06-291-2/+2
| | | | | | | | | | | | | | | | incorrect for toJavaRDD, … ## What changes were proposed in this pull request? Change the return type mentioned in the JavaDoc for `toJavaRDD` / `javaRDD` to match the actual return type & be consistent with the scala rdd return type. ## How was this patch tested? Docs only change. Author: Holden Karau <holden@us.ibm.com> Closes #13954 from holdenk/trivial-streaming-tojavardd-doc-fix.
* [SPARK-16271][SQL] Implement Hive's UDFXPathUtilpetermaxlee2016-06-283-0/+394
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch ports Hive's UDFXPathUtil over to Spark, which can be used to implement xpath functionality in Spark in the near future. ## How was this patch tested? Added two new test suites UDFXPathUtilSuite and ReusableStringReaderSuite. They have been ported over from Hive (but rewritten in Scala in order to leverage ScalaTest). Author: petermaxlee <petermaxlee@gmail.com> Closes #13961 from petermaxlee/xpath.
* [SPARK-16248][SQL] Whitelist the list of Hive fallback functionsReynold Xin2016-06-284-50/+40
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch removes the blind fallback into Hive for functions. Instead, it creates a whitelist and adds only a small number of functions to the whitelist, i.e. the ones we intend to support in the long run in Spark. ## How was this patch tested? Updated tests to reflect the change. Author: Reynold Xin <rxin@databricks.com> Closes #13939 from rxin/hive-whitelist.
* [MINOR][DOCS][STRUCTURED STREAMING] Minor doc fixes around `DataFrameWriter` ↵Burak Yavuz2016-06-287-15/+15
| | | | | | | | | | | | and `DataStreamWriter` ## What changes were proposed in this pull request? Fixes a couple old references to `DataFrameWriter.startStream` to `DataStreamWriter.start Author: Burak Yavuz <brkyvz@gmail.com> Closes #13952 from brkyvz/minor-doc-fix.
* [SPARK-16100][SQL] fix bug when use Map as the buffer type of AggregatorWenchen Fan2016-06-292-11/+32
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The root cause is in `MapObjects`. Its parameter `loopVar` is not declared as child, but sometimes can be same with `lambdaFunction`(e.g. the function that takes `loopVar` and produces `lambdaFunction` may be `identity`), which is a child. This brings trouble when call `withNewChildren`, it may mistakenly treat `loopVar` as a child and cause `IndexOutOfBoundsException: 0` later. This PR fixes this bug by simply pulling out the paremters from `LambdaVariable` and pass them to `MapObjects` directly. ## How was this patch tested? new test in `DatasetAggregatorSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #13835 from cloud-fan/map-objects.
* [SPARK-16236][SQL] Add Path Option back to Load API in DataFrameReadergatorsmile2016-06-282-1/+29
| | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? koertkuipers identified the PR https://github.com/apache/spark/pull/13727/ changed the behavior of `load` API. After the change, the `load` API does not add the value of `path` into the `options`. Thank you! This PR is to add the option `path` back to `load()` API in `DataFrameReader`, if and only if users specify one and only one `path` in the `load` API. For example, users can see the `path` option after the following API call, ```Scala spark.read .format("parquet") .load("/test") ``` #### How was this patch tested? Added test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #13933 from gatorsmile/optionPath.
* [SPARK-16181][SQL] outer join with isNull filter may return wrong resultWenchen Fan2016-06-282-0/+17
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The root cause is: the output attributes of outer join are derived from its children, while they are actually different attributes(outer join can return null). We have already added some special logic to handle it, e.g. `PushPredicateThroughJoin` won't push down predicates through outer join side, `FixNullability`. This PR adds one more special logic in `FoldablePropagation`. ## How was this patch tested? new test in `DataFrameSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #13884 from cloud-fan/bug.
* [SPARK-16128][SQL] Allow setting length of characters to be truncated to, in ↵Prashant Sharma2016-06-283-11/+65
| | | | | | | | | | | | | | | | | | Dataset.show function. ## What changes were proposed in this pull request? Allowing truncate to a specific number of character is convenient at times, especially while operating from the REPL. Sometimes those last few characters make all the difference, and showing everything brings in whole lot of noise. ## How was this patch tested? Existing tests. + 1 new test in DataFrameSuite. For SparkR and pyspark, existing tests and manual testing. Author: Prashant Sharma <prashsh1@in.ibm.com> Author: Prashant Sharma <prashant@apache.org> Closes #13839 from ScrapCodes/add_truncateTo_DF.show.
* [SPARK-16202][SQL][DOC] Correct The Description of ↵gatorsmile2016-06-271-3/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | CreatableRelationProvider's createRelation #### What changes were proposed in this pull request? The API description of `createRelation` in `CreatableRelationProvider` is misleading. The current description only expects users to return the relation. ```Scala trait CreatableRelationProvider { def createRelation( sqlContext: SQLContext, mode: SaveMode, parameters: Map[String, String], data: DataFrame): BaseRelation } ``` However, the major goal of this API should also include saving the `DataFrame`. Since this API is critical for Data Source API developers, this PR is to correct the description. #### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #13903 from gatorsmile/readUnderscoreFiles.
* [SPARK-16221][SQL] Redirect Parquet JUL logger via SLF4J for WRITE operationsDongjoon Hyun2016-06-281-5/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? [SPARK-8118](https://github.com/apache/spark/pull/8196) implements redirecting Parquet JUL logger via SLF4J, but it is currently applied only when READ operations occurs. If users use only WRITE operations, there occurs many Parquet logs. This PR makes the redirection work on WRITE operations, too. **Before** ```scala scala> spark.range(10).write.format("parquet").mode("overwrite").save("/tmp/p") SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder". SLF4J: Defaulting to no-operation (NOP) logger implementation SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further details. Jun 26, 2016 9:04:38 PM INFO: org.apache.parquet.hadoop.codec.CodecConfig: Compression: SNAPPY ............ about 70 lines Parquet Log ............. scala> spark.range(10).write.format("parquet").mode("overwrite").save("/tmp/p") ............ about 70 lines Parquet Log ............. ``` **After** ```scala scala> spark.range(10).write.format("parquet").mode("overwrite").save("/tmp/p") SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder". SLF4J: Defaulting to no-operation (NOP) logger implementation SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further details. scala> spark.range(10).write.format("parquet").mode("overwrite").save("/tmp/p") ``` This PR also fixes some typos. ## How was this patch tested? Manual. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13918 from dongjoon-hyun/SPARK-16221.
* [SPARK-16220][SQL] Add scope to show functionsHerman van Hovell2016-06-279-24/+70
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Spark currently shows all functions when issue a `SHOW FUNCTIONS` command. This PR refines the `SHOW FUNCTIONS` command by allowing users to select all functions, user defined function or system functions. The following syntax can be used: **ALL** (default) ```SHOW FUNCTIONS``` ```SHOW ALL FUNCTIONS``` **SYSTEM** ```SHOW SYSTEM FUNCTIONS``` **USER** ```SHOW USER FUNCTIONS``` ## How was this patch tested? Updated tests and added tests to the DDLSuite Author: Herman van Hovell <hvanhovell@databricks.com> Closes #13929 from hvanhovell/SPARK-16220.
* [SPARK-16220][SQL] Revert Change to Bring Back SHOW FUNCTIONS FunctionalityBill Chambers2016-06-274-8/+11
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - Fix tests regarding show functions functionality - Revert `catalog.ListFunctions` and `SHOW FUNCTIONS` to return to `Spark 1.X` functionality. Cherry picked changes from this PR: https://github.com/apache/spark/pull/13413/files ## How was this patch tested? Unit tests. Author: Bill Chambers <bill@databricks.com> Author: Bill Chambers <wchambers@ischool.berkeley.edu> Closes #13916 from anabranch/master.
* [SPARK-16135][SQL] Remove hashCode and euqals in ArrayBasedMapDataTakeshi YAMAMURO2016-06-276-23/+76
| | | | | | | | | | | | ## What changes were proposed in this pull request? This pr is to remove `hashCode` and `equals` in `ArrayBasedMapData` because the type cannot be used as join keys, grouping keys, or in equality tests. ## How was this patch tested? Add a new test suite `MapDataSuite` for comparison tests. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #13847 from maropu/UnsafeMapTest.
* [SPARK-10591][SQL][TEST] Add a testcase to ensure if `checkAnswer` handles ↵Dongjoon Hyun2016-06-271-0/+7
| | | | | | | | | | | | | | | | map correctly ## What changes were proposed in this pull request? This PR adds a testcase to ensure if `checkAnswer` handles Map type correctly. ## How was this patch tested? Pass the jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13913 from dongjoon-hyun/SPARK-10591.
* [SPARK-16184][SPARKR] conf API for SparkSessionFelix Cheung2016-06-261-0/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add `conf` method to get Runtime Config from SparkSession ## How was this patch tested? unit tests, manual tests This is how it works in sparkR shell: ``` SparkSession available as 'spark'. > conf() $hive.metastore.warehouse.dir [1] "file:/opt/spark-2.0.0-bin-hadoop2.6/R/spark-warehouse" $spark.app.id [1] "local-1466749575523" $spark.app.name [1] "SparkR" $spark.driver.host [1] "10.0.2.1" $spark.driver.port [1] "45629" $spark.executorEnv.LD_LIBRARY_PATH [1] "$LD_LIBRARY_PATH:/usr/lib/R/lib:/usr/lib/x86_64-linux-gnu:/usr/lib/jvm/default-java/jre/lib/amd64/server" $spark.executor.id [1] "driver" $spark.home [1] "/opt/spark-2.0.0-bin-hadoop2.6" $spark.master [1] "local[*]" $spark.sql.catalogImplementation [1] "hive" $spark.submit.deployMode [1] "client" > conf("spark.master") $spark.master [1] "local[*]" ``` Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #13885 from felixcheung/rconf.
* [SPARK-15958] Make initial buffer size for the Sorter configurableSital Kedia2016-06-252-3/+8
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently the initial buffer size in the sorter is hard coded inside the code and is too small for large workload. As a result, the sorter spends significant time expanding the buffer size and copying the data. It would be useful to have it configurable. ## How was this patch tested? Tested by running a job on the cluster. Author: Sital Kedia <skedia@fb.com> Closes #13699 from sitalkedia/config_sort_buffer_upstream.
* [SPARK-16186] [SQL] Support partition batch pruning with `IN` predicate in ↵Dongjoon Hyun2016-06-242-1/+26
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | InMemoryTableScanExec ## What changes were proposed in this pull request? One of the most frequent usage patterns for Spark SQL is using **cached tables**. This PR improves `InMemoryTableScanExec` to handle `IN` predicate efficiently by pruning partition batches. Of course, the performance improvement varies over the queries and the datasets. But, for the following simple query, the query duration in Spark UI goes from 9 seconds to 50~90ms. It's about over 100 times faster. **Before** ```scala $ bin/spark-shell --driver-memory 6G scala> val df = spark.range(2000000000) scala> df.createOrReplaceTempView("t") scala> spark.catalog.cacheTable("t") scala> sql("select id from t where id = 1").collect() // About 2 mins scala> sql("select id from t where id = 1").collect() // less than 90ms scala> sql("select id from t where id in (1,2,3)").collect() // 9 seconds ``` **After** ```scala scala> sql("select id from t where id in (1,2,3)").collect() // less than 90ms ``` This PR has impacts over 35 queries of TPC-DS if the tables are cached. Note that this optimization is applied for `IN`. To apply `IN` predicate having more than 10 items, `spark.sql.optimizer.inSetConversionThreshold` option should be increased. ## How was this patch tested? Pass the Jenkins tests (including new testcases). Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13887 from dongjoon-hyun/SPARK-16186.
* [SPARK-16192][SQL] Add type checks in CollectSetTakeshi YAMAMURO2016-06-243-2/+21
| | | | | | | | | | | | | ## What changes were proposed in this pull request? `CollectSet` cannot have map-typed data because MapTypeData does not implement `equals`. So, this pr is to add type checks in `CheckAnalysis`. ## How was this patch tested? Added tests to check failures when we found map-typed data in `CollectSet`. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #13892 from maropu/SPARK-16192.
* [SPARK-16195][SQL] Allow users to specify empty over clause in window ↵Dilip Biswal2016-06-243-1/+30
| | | | | | | | | | | | | | | | | | | | | | | | | | expressions through dataset API ## What changes were proposed in this pull request? Allow to specify empty over clause in window expressions through dataset API In SQL, its allowed to specify an empty OVER clause in the window expression. ```SQL select area, sum(product) over () as c from windowData where product > 3 group by area, product having avg(month) > 0 order by avg(month), product ``` In this case the analytic function sum is presented based on all the rows of the result set Currently its not allowed through dataset API and is handled in this PR. ## How was this patch tested? Added a new test in DataframeWindowSuite Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #13897 from dilipbiswal/spark-empty-over.
* [SPARK-16173] [SQL] Can't join describe() of DataFrame in Scala 2.10Dongjoon Hyun2016-06-241-1/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes `DataFrame.describe()` by forcing materialization to make the `Seq` serializable. Currently, `describe()` of DataFrame throws `Task not serializable` Spark exceptions when joining in Scala 2.10. ## How was this patch tested? Manual. (After building with Scala 2.10, test on `bin/spark-shell` and `bin/pyspark`.) Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13900 from dongjoon-hyun/SPARK-16173.
* Revert "[SPARK-16186] [SQL] Support partition batch pruning with `IN` ↵Davies Liu2016-06-242-26/+1
| | | | | | predicate in InMemoryTableScanExec" This reverts commit a65bcbc27dcd9b3053cb13c5d67251c8d48f4397.
* [SPARK-16186] [SQL] Support partition batch pruning with `IN` predicate in ↵Dongjoon Hyun2016-06-242-1/+26
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | InMemoryTableScanExec ## What changes were proposed in this pull request? One of the most frequent usage patterns for Spark SQL is using **cached tables**. This PR improves `InMemoryTableScanExec` to handle `IN` predicate efficiently by pruning partition batches. Of course, the performance improvement varies over the queries and the datasets. But, for the following simple query, the query duration in Spark UI goes from 9 seconds to 50~90ms. It's about over 100 times faster. **Before** ```scala $ bin/spark-shell --driver-memory 6G scala> val df = spark.range(2000000000) scala> df.createOrReplaceTempView("t") scala> spark.catalog.cacheTable("t") scala> sql("select id from t where id = 1").collect() // About 2 mins scala> sql("select id from t where id = 1").collect() // less than 90ms scala> sql("select id from t where id in (1,2,3)").collect() // 9 seconds ``` **After** ```scala scala> sql("select id from t where id in (1,2,3)").collect() // less than 90ms ``` This PR has impacts over 35 queries of TPC-DS if the tables are cached. Note that this optimization is applied for `IN`. To apply `IN` predicate having more than 10 items, `spark.sql.optimizer.inSetConversionThreshold` option should be increased. ## How was this patch tested? Pass the Jenkins tests (including new testcases). Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13887 from dongjoon-hyun/SPARK-16186.
* [SPARK-16179][PYSPARK] fix bugs for Python udf in generateDavies Liu2016-06-241-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fix the bug when Python UDF is used in explode (generator), GenerateExec requires that all the attributes in expressions should be resolvable from children when creating, we should replace the children first, then replace it's expressions. ``` >>> df.select(explode(f(*df))).show() Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/home/vlad/dev/spark/python/pyspark/sql/dataframe.py", line 286, in show print(self._jdf.showString(n, truncate)) File "/home/vlad/dev/spark/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__ File "/home/vlad/dev/spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/home/vlad/dev/spark/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 312, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o52.showString. : org.apache.spark.sql.catalyst.errors.package$TreeNodeException: makeCopy, tree: Generate explode(<lambda>(_1#0L)), false, false, [col#15L] +- Scan ExistingRDD[_1#0L] at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50) at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:387) at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:69) at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:45) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsDown(QueryPlan.scala:177) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressions(QueryPlan.scala:144) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$.org$apache$spark$sql$execution$python$ExtractPythonUDFs$$extract(ExtractPythonUDFs.scala:153) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$$anonfun$apply$2.applyOrElse(ExtractPythonUDFs.scala:114) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$$anonfun$apply$2.applyOrElse(ExtractPythonUDFs.scala:113) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:301) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:301) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69) at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:300) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:298) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:298) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319) at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:298) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$.apply(ExtractPythonUDFs.scala:113) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$.apply(ExtractPythonUDFs.scala:93) at org.apache.spark.sql.execution.QueryExecution$$anonfun$prepareForExecution$1.apply(QueryExecution.scala:95) at org.apache.spark.sql.execution.QueryExecution$$anonfun$prepareForExecution$1.apply(QueryExecution.scala:95) at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:124) at scala.collection.immutable.List.foldLeft(List.scala:84) at org.apache.spark.sql.execution.QueryExecution.prepareForExecution(QueryExecution.scala:95) at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:85) at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:85) at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2557) at org.apache.spark.sql.Dataset.head(Dataset.scala:1923) at org.apache.spark.sql.Dataset.take(Dataset.scala:2138) at org.apache.spark.sql.Dataset.showString(Dataset.scala:239) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:237) at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357) at py4j.Gateway.invoke(Gateway.java:280) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:211) at java.lang.Thread.run(Thread.java:745) Caused by: java.lang.reflect.InvocationTargetException at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1$$anonfun$apply$13.apply(TreeNode.scala:413) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1$$anonfun$apply$13.apply(TreeNode.scala:413) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1.apply(TreeNode.scala:412) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1.apply(TreeNode.scala:387) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49) ... 42 more Caused by: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: pythonUDF0#20 at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:87) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:279) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:279) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:278) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:284) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:268) at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:87) at org.apache.spark.sql.execution.GenerateExec.<init>(GenerateExec.scala:63) ... 52 more Caused by: java.lang.RuntimeException: Couldn't find pythonUDF0#20 in [_1#0L] at scala.sys.package$.error(package.scala:27) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:94) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:88) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49) ... 67 more ``` ## How was this patch tested? Added regression tests. Author: Davies Liu <davies@databricks.com> Closes #13883 from davies/udf_in_generate.
* [SQL][MINOR] Simplify data source predicate filter translation.Reynold Xin2016-06-241-44/+25
| | | | | | | | | | | | ## What changes were proposed in this pull request? This is a small patch to rewrite the predicate filter translation in DataSourceStrategy. The original code used excessive functional constructs (e.g. unzip) and was very difficult to understand. ## How was this patch tested? Should be covered by existing tests. Author: Reynold Xin <rxin@databricks.com> Closes #13889 from rxin/simplify-predicate-filter.
* [SPARK-16129][CORE][SQL] Eliminate direct use of commons-lang classes in ↵Sean Owen2016-06-249-38/+32
| | | | | | | | | | | | | | | | favor of commons-lang3 ## What changes were proposed in this pull request? Replace use of `commons-lang` in favor of `commons-lang3` and forbid the former via scalastyle; remove `NotImplementedException` from `comons-lang` in favor of JDK `UnsupportedOperationException` ## How was this patch tested? Jenkins tests Author: Sean Owen <sowen@cloudera.com> Closes #13843 from srowen/SPARK-16129.
* [SPARK-13709][SQL] Initialize deserializer with both table and partition ↵Cheng Lian2016-06-232-1/+97
| | | | | | | | | | | | | | | | | | properties when reading partitioned tables ## What changes were proposed in this pull request? When reading partitions of a partitioned Hive SerDe table, we only initializes the deserializer using partition properties. However, for SerDes like `AvroSerDe`, essential properties (e.g. Avro schema information) may be defined in table properties. We should merge both table properties and partition properties before initializing the deserializer. Note that an individual partition may have different properties than the one defined in the table properties (e.g. partitions within a table can have different SerDes). Thus, for any property key defined in both partition and table properties, the value set in partition properties wins. ## How was this patch tested? New test case added in `QueryPartitionSuite`. Author: Cheng Lian <lian@databricks.com> Closes #13865 from liancheng/spark-13709-partitioned-avro-table.
* [SQL][MINOR] ParserUtils.operationNotAllowed should throw exception directlyWenchen Fan2016-06-232-33/+33
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? It's weird that `ParserUtils.operationNotAllowed` returns an exception and the caller throw it. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #13874 from cloud-fan/style.
* [SPARK-16123] Avoid NegativeArraySizeException while reserving additional ↵Sameer Agarwal2016-06-234-14/+47
| | | | | | | | | | | | | | | | capacity in VectorizedColumnReader ## What changes were proposed in this pull request? This patch fixes an overflow bug in vectorized parquet reader where both off-heap and on-heap variants of `ColumnVector.reserve()` can unfortunately overflow while reserving additional capacity during reads. ## How was this patch tested? Manual Tests Author: Sameer Agarwal <sameer@databricks.com> Closes #13832 from sameeragarwal/negative-array.
* [SPARK-16165][SQL] Fix the update logic for InMemoryTableScanExec.readBatchesDongjoon Hyun2016-06-242-3/+18
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, `readBatches` accumulator of `InMemoryTableScanExec` is updated only when `spark.sql.inMemoryColumnarStorage.partitionPruning` is true. Although this metric is used for only testing purpose, we had better have correct metric without considering SQL options. ## How was this patch tested? Pass the Jenkins tests (including a new testcase). Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13870 from dongjoon-hyun/SPARK-16165.
* [SPARK-15443][SQL] Fix 'explain' for streaming DatasetShixiong Zhu2016-06-238-2/+142
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - Fix the `explain` command for streaming Dataset/DataFrame. E.g., ``` == Parsed Logical Plan == 'SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- 'MapElements <function1>, obj#6: java.lang.String +- 'DeserializeToObject unresolveddeserializer(createexternalrow(getcolumnbyordinal(0, StringType).toString, StructField(value,StringType,true))), obj#5: org.apache.spark.sql.Row +- Filter <function1>.apply +- StreamingRelation FileSource[/Users/zsx/stream], [value#0] == Analyzed Logical Plan == value: string SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- MapElements <function1>, obj#6: java.lang.String +- DeserializeToObject createexternalrow(value#0.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- Filter <function1>.apply +- StreamingRelation FileSource[/Users/zsx/stream], [value#0] == Optimized Logical Plan == SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- MapElements <function1>, obj#6: java.lang.String +- DeserializeToObject createexternalrow(value#0.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- Filter <function1>.apply +- StreamingRelation FileSource[/Users/zsx/stream], [value#0] == Physical Plan == *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- *MapElements <function1>, obj#6: java.lang.String +- *DeserializeToObject createexternalrow(value#0.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- *Filter <function1>.apply +- StreamingRelation FileSource[/Users/zsx/stream], [value#0] ``` - Add `StreamingQuery.explain` to display the last execution plan. E.g., ``` == Parsed Logical Plan == SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- MapElements <function1>, obj#6: java.lang.String +- DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- Filter <function1>.apply +- Relation[value#12] text == Analyzed Logical Plan == value: string SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- MapElements <function1>, obj#6: java.lang.String +- DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- Filter <function1>.apply +- Relation[value#12] text == Optimized Logical Plan == SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- MapElements <function1>, obj#6: java.lang.String +- DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- Filter <function1>.apply +- Relation[value#12] text == Physical Plan == *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- *MapElements <function1>, obj#6: java.lang.String +- *DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- *Filter <function1>.apply +- *Scan text [value#12] Format: org.apache.spark.sql.execution.datasources.text.TextFileFormat1836ab91, InputPaths: file:/Users/zsx/stream/a.txt, file:/Users/zsx/stream/b.txt, file:/Users/zsx/stream/c.txt, PushedFilters: [], ReadSchema: struct<value:string> ``` ## How was this patch tested? The added unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #13815 from zsxwing/sdf-explain.
* [SPARK-16164][SQL] Update `CombineFilters` to try to construct predicates ↵Dongjoon Hyun2016-06-232-1/+19
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | with child predicate first ## What changes were proposed in this pull request? This PR changes `CombineFilters` to compose the final predicate condition by using (`child predicate` AND `parent predicate`) instead of (`parent predicate` AND `child predicate`). This is a best effort approach. Some other optimization rules may destroy this order by reorganizing conjunctive predicates. **Reported Error Scenario** Chris McCubbin reported a bug when he used StringIndexer in an ML pipeline with additional filters. It seems that during filter pushdown, we changed the ordering in the logical plan. ```scala import org.apache.spark.ml.feature._ val df1 = (0 until 3).map(_.toString).toDF val indexer = new StringIndexer() .setInputCol("value") .setOutputCol("idx") .setHandleInvalid("skip") .fit(df1) val df2 = (0 until 5).map(_.toString).toDF val predictions = indexer.transform(df2) predictions.show() // this is okay predictions.where('idx > 2).show() // this will throw an exception ``` Please see the notebook at https://databricks-prod-cloudfront.cloud.databricks.com/public/4027ec902e239c93eaaa8714f173bcfc/1233855/2159162931615821/588180/latest.html for error messages. ## How was this patch tested? Pass the Jenkins tests (including a new testcase). Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13872 from dongjoon-hyun/SPARK-16164.
* [SPARK-16163] [SQL] Cache the statistics for logical plansDavies Liu2016-06-233-58/+30
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This calculation of statistics is not trivial anymore, it could be very slow on large query (for example, TPC-DS Q64 took several minutes to plan). During the planning of a query, the statistics of any logical plan should not change (even InMemoryRelation), so we should use `lazy val` to cache the statistics. For InMemoryRelation, the statistics could be updated after materialization, it's only useful when used in another query (before planning), because once we finished the planning, the statistics will not be used anymore. ## How was this patch tested? Testsed with TPC-DS Q64, it could be planned in a second after the patch. Author: Davies Liu <davies@databricks.com> Closes #13871 from davies/fix_statistics.
* [SPARK-16116][SQL] ConsoleSink should not require checkpointLocationShixiong Zhu2016-06-232-0/+18
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? When the user uses `ConsoleSink`, we should use a temp location if `checkpointLocation` is not specified. ## How was this patch tested? The added unit test. Author: Shixiong Zhu <shixiong@databricks.com> Closes #13817 from zsxwing/console-checkpoint.
* [SPARK-16162] Remove dead code OrcTableScan.Brian Cho2016-06-221-66/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? SPARK-14535 removed all calls to class OrcTableScan. This removes the dead code. ## How was this patch tested? Existing unit tests. Author: Brian Cho <bcho@fb.com> Closes #13869 from dafrista/clean-up-orctablescan.
* [SQL][MINOR] Fix minor formatting issues in SHOW CREATE TABLE outputCheng Lian2016-06-221-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes two minor formatting issues appearing in `SHOW CREATE TABLE` output. Before: ``` CREATE EXTERNAL TABLE ... ... WITH SERDEPROPERTIES ('serialization.format' = '1' ) ... TBLPROPERTIES ('avro.schema.url' = '/tmp/avro/test.avsc', 'transient_lastDdlTime' = '1466638180') ``` After: ``` CREATE EXTERNAL TABLE ... ... WITH SERDEPROPERTIES ( 'serialization.format' = '1' ) ... TBLPROPERTIES ( 'avro.schema.url' = '/tmp/avro/test.avsc', 'transient_lastDdlTime' = '1466638180' ) ``` ## How was this patch tested? Manually tested. Author: Cheng Lian <lian@databricks.com> Closes #13864 from liancheng/show-create-table-format-fix.
* [SPARK-15230][SQL] distinct() does not handle column name with dot properlybomeng2016-06-232-1/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? When table is created with column name containing dot, distinct() will fail to run. For example, ```scala val rowRDD = sparkContext.parallelize(Seq(Row(1), Row(1), Row(2))) val schema = StructType(Array(StructField("column.with.dot", IntegerType, nullable = false))) val df = spark.createDataFrame(rowRDD, schema) ``` running the following will have no problem: ```scala df.select(new Column("`column.with.dot`")) ``` but running the query with additional distinct() will cause exception: ```scala df.select(new Column("`column.with.dot`")).distinct() ``` The issue is that distinct() will try to resolve the column name, but the column name in the schema does not have backtick with it. So the solution is to add the backtick before passing the column name to resolve(). ## How was this patch tested? Added a new test case. Author: bomeng <bmeng@us.ibm.com> Closes #13140 from bomeng/SPARK-15230.