aboutsummaryrefslogtreecommitdiff
path: root/sql
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-10845] [SQL] Makes spark.sql.hive.version a SQLConfEntryCheng Lian2015-09-263-1/+32
| | | | | | | | | | When refactoring SQL options from plain strings to the strongly typed `SQLConfEntry`, `spark.sql.hive.version` wasn't migrated, and doesn't show up in the result of `SET -v`, as `SET -v` only shows public `SQLConfEntry` instances. This affects compatibility with Simba ODBC driver. This PR migrates this SQL option as a `SQLConfEntry` to fix this issue. Author: Cheng Lian <lian@databricks.com> Closes #8925 from liancheng/spark-10845/hive-version-conf.
* [SPARK-9852] Let reduce tasks fetch multiple map output partitionsMatei Zaharia2015-09-241-0/+6
| | | | | | | | | | | | | This makes two changes: - Allow reduce tasks to fetch multiple map output partitions -- this is a pretty small change to HashShuffleFetcher - Move shuffle locality computation out of DAGScheduler and into ShuffledRDD / MapOutputTracker; this was needed because the code in DAGScheduler wouldn't work for RDDs that fetch multiple map output partitions from each reduce task I also added an AdaptiveSchedulingSuite that creates RDDs depending on multiple map output partitions. Author: Matei Zaharia <matei@databricks.com> Closes #8844 from mateiz/spark-9852.
* [SPARK-10705] [SQL] Avoid using external rows in DataFrame.toJSONLiang-Chi Hsieh2015-09-243-85/+3
| | | | | | | | | | JIRA: https://issues.apache.org/jira/browse/SPARK-10705 As described in the JIRA ticket, `DataFrame.toJSON` uses `DataFrame.mapPartitions`, which converts internal rows to external rows. We should use `queryExecution.toRdd.mapPartitions` that directly uses internal rows for better performance. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #8865 from viirya/df-tojson-internalrow.
* [SPARK-10765] [SQL] use new aggregate interface for hive UDAFWenchen Fan2015-09-246-86/+129
| | | | | | Author: Wenchen Fan <cloud0fan@163.com> Closes #8874 from cloud-fan/hive-agg.
* [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array ↵Andrew Or2015-09-232-13/+4
| | | | | | | | | | | | | | (round 2) This patch reverts most of the changes in a previous fix #8827. The real cause of the issue is that in `TungstenAggregate`'s prepare method we only reserve 1 page, but later when we switch to sort-based aggregation we try to acquire 1 page AND a pointer array. The longer-term fix should be to reserve also the pointer array, but for now ***we will simply not track the pointer array***. (Note that elsewhere we already don't track the pointer array, e.g. [here](https://github.com/apache/spark/blob/a18208047f06a4244703c17023bb20cbe1f59d73/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java#L88)) Note: This patch reuses the unit test added in #8827 so it doesn't show up in the diff. Author: Andrew Or <andrew@databricks.com> Closes #8888 from andrewor14/dont-track-pointer-array.
* [SPARK-10731] [SQL] Delegate to Scala's DataFrame.take implementation in ↵Reynold Xin2015-09-232-10/+20
| | | | | | | | | | | | Python DataFrame. Python DataFrame.head/take now requires scanning all the partitions. This pull request changes them to delegate the actual implementation to Scala DataFrame (by calling DataFrame.take). This is more of a hack for fixing this issue in 1.5.1. A more proper fix is to change executeCollect and executeTake to return InternalRow rather than Row, and thus eliminate the extra round-trip conversion. Author: Reynold Xin <rxin@databricks.com> Closes #8876 from rxin/SPARK-10731.
* [SPARK-10403] Allow UnsafeRowSerializer to work with tungsten-sort ↵Josh Rosen2015-09-232-18/+27
| | | | | | | | | | ShuffleManager This patch attempts to fix an issue where Spark SQL's UnsafeRowSerializer was incompatible with the `tungsten-sort` ShuffleManager. Author: Josh Rosen <joshrosen@databricks.com> Closes #8873 from JoshRosen/SPARK-10403.
* [SPARK-10310] [SQL] Fixes script transformation field/line delimitersZhichao Li2015-09-225-22/+152
| | | | | | | | | | | | **Please attribute this PR to `Zhichao Li <zhichao.liintel.com>`.** This PR is based on PR #8476 authored by zhichao-li. It fixes SPARK-10310 by adding field delimiter SerDe property to the default `LazySimpleSerDe`, and enabling default record reader/writer classes. Currently, we only support `LazySimpleSerDe`, used together with `TextRecordReader` and `TextRecordWriter`, and don't support customizing record reader/writer using `RECORDREADER`/`RECORDWRITER` clauses. This should be addressed in separate PR(s). Author: Cheng Lian <lian@databricks.com> Closes #8860 from liancheng/spark-10310/fix-script-trans-delimiters.
* [SPARK-10714] [SPARK-8632] [SPARK-10685] [SQL] Refactor Python UDF handlingReynold Xin2015-09-221-34/+46
| | | | | | | | | | | | | | | | This patch refactors Python UDF handling: 1. Extract the per-partition Python UDF calling logic from PythonRDD into a PythonRunner. PythonRunner itself expects iterator as input/output, and thus has no dependency on RDD. This way, we can use PythonRunner directly in a mapPartitions call, or in the future in an environment without RDDs. 2. Use PythonRunner in Spark SQL's BatchPythonEvaluation. 3. Updated BatchPythonEvaluation to only use its input once, rather than twice. This should fix Python UDF performance regression in Spark 1.5. There are a number of small cleanups I wanted to do when I looked at the code, but I kept most of those out so the diff looks small. This basically implements the approach in https://github.com/apache/spark/pull/8833, but with some code moving around so the correctness doesn't depend on the inner workings of Spark serialization and task execution. Author: Reynold Xin <rxin@databricks.com> Closes #8835 from rxin/python-iter-refactor.
* [SPARK-10737] [SQL] When using UnsafeRows, SortMergeJoin may return wrong ↵Yin Huai2015-09-224-5/+59
| | | | | | | | | | results https://issues.apache.org/jira/browse/SPARK-10737 Author: Yin Huai <yhuai@databricks.com> Closes #8854 from yhuai/SMJBug.
* [SPARK-10672] [SQL] Do not fail when we cannot save the metadata of a data ↵Yin Huai2015-09-221-51/+50
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | source table in a hive compatible way https://issues.apache.org/jira/browse/SPARK-10672 With changes in this PR, we will fallback to same the metadata of a table in Spark SQL specific way if we fail to save it in a hive compatible way (Hive throws an exception because of its internal restrictions, e.g. binary and decimal types cannot be saved to parquet if the metastore is running Hive 0.13). I manually tested the fix with the following test in `DataSourceWithHiveMetastoreCatalogSuite` (`spark.sql.hive.metastore.version=0.13` and `spark.sql.hive.metastore.jars`=`maven`). ``` test(s"fail to save metadata of a parquet table in hive 0.13") { withTempPath { dir => withTable("t") { val path = dir.getCanonicalPath sql( s"""CREATE TABLE t USING $provider |OPTIONS (path '$path') |AS SELECT 1 AS d1, cast("val_1" as binary) AS d2 """.stripMargin) sql( s"""describe formatted t """.stripMargin).collect.foreach(println) sqlContext.table("t").show } } } } ``` Without this fix, we will fail with the following error. ``` org.apache.hadoop.hive.ql.metadata.HiveException: java.lang.UnsupportedOperationException: Unknown field type: binary at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:619) at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:576) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$createTable$1.apply$mcV$sp(ClientWrapper.scala:359) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$createTable$1.apply(ClientWrapper.scala:357) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$createTable$1.apply(ClientWrapper.scala:357) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$withHiveState$1.apply(ClientWrapper.scala:256) at org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:211) at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:248) at org.apache.spark.sql.hive.client.ClientWrapper.createTable(ClientWrapper.scala:357) at org.apache.spark.sql.hive.HiveMetastoreCatalog.createDataSourceTable(HiveMetastoreCatalog.scala:358) at org.apache.spark.sql.hive.execution.CreateMetastoreDataSourceAsSelect.run(commands.scala:285) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:57) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:57) at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:69) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:140) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:138) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:138) at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:58) at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:58) at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:144) at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:129) at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:725) at org.apache.spark.sql.test.SQLTestUtils$$anonfun$sql$1.apply(SQLTestUtils.scala:56) at org.apache.spark.sql.test.SQLTestUtils$$anonfun$sql$1.apply(SQLTestUtils.scala:56) at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1$$anonfun$apply$mcV$sp$2$$anonfun$apply$2.apply$mcV$sp(HiveMetastoreCatalogSuite.scala:165) at org.apache.spark.sql.test.SQLTestUtils$class.withTable(SQLTestUtils.scala:150) at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.withTable(HiveMetastoreCatalogSuite.scala:52) at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1$$anonfun$apply$mcV$sp$2.apply(HiveMetastoreCatalogSuite.scala:162) at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1$$anonfun$apply$mcV$sp$2.apply(HiveMetastoreCatalogSuite.scala:161) at org.apache.spark.sql.test.SQLTestUtils$class.withTempPath(SQLTestUtils.scala:125) at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.withTempPath(HiveMetastoreCatalogSuite.scala:52) at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1.apply$mcV$sp(HiveMetastoreCatalogSuite.scala:161) at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1.apply(HiveMetastoreCatalogSuite.scala:161) at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1.apply(HiveMetastoreCatalogSuite.scala:161) at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) at org.scalatest.Transformer.apply(Transformer.scala:22) at org.scalatest.Transformer.apply(Transformer.scala:20) at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:42) at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) at org.scalatest.FunSuite.runTest(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401) at scala.collection.immutable.List.foreach(List.scala:318) at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396) at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483) at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208) at org.scalatest.FunSuite.runTests(FunSuite.scala:1555) at org.scalatest.Suite$class.run(Suite.scala:1424) at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at org.scalatest.SuperEngine.runImpl(Engine.scala:545) at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212) at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.org$scalatest$BeforeAndAfterAll$$super$run(HiveMetastoreCatalogSuite.scala:52) at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257) at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256) at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.run(HiveMetastoreCatalogSuite.scala:52) at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462) at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671) at sbt.ForkMain$Run$2.call(ForkMain.java:294) at sbt.ForkMain$Run$2.call(ForkMain.java:284) at java.util.concurrent.FutureTask.run(FutureTask.java:262) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) Caused by: java.lang.UnsupportedOperationException: Unknown field type: binary at org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector.getObjectInspector(ArrayWritableObjectInspector.java:108) at org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector.<init>(ArrayWritableObjectInspector.java:60) at org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe.initialize(ParquetHiveSerDe.java:113) at org.apache.hadoop.hive.metastore.MetaStoreUtils.getDeserializer(MetaStoreUtils.java:339) at org.apache.hadoop.hive.ql.metadata.Table.getDeserializerFromMetaStore(Table.java:288) at org.apache.hadoop.hive.ql.metadata.Table.checkValidity(Table.java:194) at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:597) ... 76 more ``` Author: Yin Huai <yhuai@databricks.com> Closes #8824 from yhuai/datasourceMetadata.
* [SPARK-10740] [SQL] handle nondeterministic expressions correctly for set ↵Wenchen Fan2015-09-223-20/+93
| | | | | | | | | | operations https://issues.apache.org/jira/browse/SPARK-10740 Author: Wenchen Fan <cloud0fan@163.com> Closes #8858 from cloud-fan/non-deter.
* [SPARK-10593] [SQL] fix resolve output of GenerateDavies Liu2015-09-224-2/+31
| | | | | | | | The output of Generate should not be resolved as Reference. Author: Davies Liu <davies@databricks.com> Closes #8755 from davies/view.
* [SQL] [MINOR] map -> foreach.Reynold Xin2015-09-221-3/+2
| | | | | | | | DataFrame.explain should use foreach to print the explain content. Author: Reynold Xin <rxin@databricks.com> Closes #8862 from rxin/map-foreach.
* [SPARK-8567] [SQL] Increase the timeout of ↵Yin Huai2015-09-221-1/+1
| | | | | | | | | | | | | | o.a.s.sql.hive.HiveSparkSubmitSuite to 5 minutes. https://issues.apache.org/jira/browse/SPARK-8567 Looks like "SPARK-8368: includes jars passed in through --jars" is pretty flaky now. Based on some history runs, the time spent on a successful run may be from 1.5 minutes to almost 3 minutes. Let's try to increase the timeout and see if we can fix this test. https://amplab.cs.berkeley.edu/jenkins/job/Spark-1.5-SBT/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.0,label=spark-test/385/testReport/junit/org.apache.spark.sql.hive/HiveSparkSubmitSuite/SPARK_8368__includes_jars_passed_in_through___jars/history/?start=25 Author: Yin Huai <yhuai@databricks.com> Closes #8850 from yhuai/SPARK-8567-anotherTry.
* [SPARK-10446][SQL] Support to specify join type when calling join with ↵Liang-Chi Hsieh2015-09-212-1/+34
| | | | | | | | | | | | usingColumns JIRA: https://issues.apache.org/jira/browse/SPARK-10446 Currently the method `join(right: DataFrame, usingColumns: Seq[String])` only supports inner join. It is more convenient to have it support other join types. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #8600 from viirya/usingcolumns_df.
* [SPARK-10419] [SQL] Adding SQLServer support for datetimeoffset types to ↵Ewan Leith2015-09-212-0/+19
| | | | | | | | | | | | JdbcDialects Reading from Microsoft SQL Server over jdbc fails when the table contains datetimeoffset types. This patch registers a SQLServer JDBC Dialect that maps datetimeoffset to a String, as Microsoft suggest. Author: Ewan Leith <ewan.leith@realitymine.com> Closes #8575 from realitymine-coordinator/sqlserver.
* [SPARK-10495] [SQL] Read date values in JSON data stored by Spark 1.5.0.Yin Huai2015-09-213-2/+152
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-10681 Author: Yin Huai <yhuai@databricks.com> Closes #8806 from yhuai/SPARK-10495.
* [SPARK-10630] [SQL] Add a createDataFrame API that takes in a java listHolden Karau2015-09-212-0/+24
| | | | | | | | It would be nice to support creating a DataFrame directly from a Java List of Row. Author: Holden Karau <holden@pigscanfly.ca> Closes #8779 from holdenk/SPARK-10630-create-DataFrame-from-Java-List.
* [SPARK-10710] Remove ability to disable spilling in core and SQLJosh Rosen2015-09-197-62/+19
| | | | | | | | | | It does not make much sense to set `spark.shuffle.spill` or `spark.sql.planner.externalSort` to false: I believe that these configurations were initially added as "escape hatches" to guard against bugs in the external operators, but these operators are now mature and well-tested. In addition, these configurations are not handled in a consistent way anymore: SQL's Tungsten codepath ignores these configurations and will continue to use spilling operators. Similarly, Spark Core's `tungsten-sort` shuffle manager does not respect `spark.shuffle.spill=false`. This pull request removes these configurations, adds warnings at the appropriate places, and deletes a large amount of code which was only used in code paths that did not support spilling. Author: Josh Rosen <joshrosen@databricks.com> Closes #8831 from JoshRosen/remove-ability-to-disable-spilling.
* [SPARK-10155] [SQL] Change SqlParser to object to avoid memory leakzsxwing2015-09-199-19/+19
| | | | | | | | | | Since `scala.util.parsing.combinator.Parsers` is thread-safe since Scala 2.10 (See [SI-4929](https://issues.scala-lang.org/browse/SI-4929)), we can change SqlParser to object to avoid memory leak. I didn't change other subclasses of `scala.util.parsing.combinator.Parsers` because there is only one instance in one SQLContext, which should not be an issue. Author: zsxwing <zsxwing@gmail.com> Closes #8357 from zsxwing/sql-memory-leak.
* [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer arrayAndrew Or2015-09-182-3/+54
| | | | | | | | | | | | | | | | When `TungstenAggregation` hits memory pressure, it switches from hash-based to sort-based aggregation in-place. However, in the process we try to allocate the pointer array for writing to the new `UnsafeExternalSorter` *before* actually freeing the memory from the hash map. This lead to the following exception: ``` java.io.IOException: Could not acquire 65536 bytes of memory at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.initializeForWriting(UnsafeExternalSorter.java:169) at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:220) at org.apache.spark.sql.execution.UnsafeKVExternalSorter.<init>(UnsafeKVExternalSorter.java:126) at org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter(UnsafeFixedWidthAggregationMap.java:257) at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.switchToSortBasedAggregation(TungstenAggregationIterator.scala:435) ``` Author: Andrew Or <andrew@databricks.com> Closes #8827 from andrewor14/allocate-pointer-array.
* [SPARK-10623] [SQL] Fixes ORC predicate push-downCheng Lian2015-09-182-34/+52
| | | | | | | | | | When pushing down a leaf predicate, ORC `SearchArgument` builder requires an extra "parent" predicate (any one among `AND`/`OR`/`NOT`) to wrap the leaf predicate. E.g., to push down `a < 1`, we must build `AND(a < 1)` instead. Fortunately, when actually constructing the `SearchArgument`, the builder will eliminate all those unnecessary wrappers. This PR is based on #8783 authored by zhzhan. I also took the chance to simply `OrcFilters` a little bit to improve readability. Author: Cheng Lian <lian@databricks.com> Closes #8799 from liancheng/spark-10623/fix-orc-ppd.
* [SPARK-10449] [SQL] Don't merge decimal types with incompatable precision or ↵Holden Karau2015-09-181-4/+13
| | | | | | | | | | scales From JIRA: Schema merging should only handle struct fields. But currently we also reconcile decimal precision and scale information. Author: Holden Karau <holden@pigscanfly.ca> Closes #8634 from holdenk/SPARK-10449-dont-merge-different-precision.
* [SPARK-10539] [SQL] Project should not be pushed down through Intersect or ↵Yijie Shen2015-09-183-30/+39
| | | | | | | | | | | | | | | Except #8742 Intersect and Except are both set operators and they use the all the columns to compare equality between rows. When pushing their Project parent down, the relations they based on would change, therefore not an equivalent transformation. JIRA: https://issues.apache.org/jira/browse/SPARK-10539 I added some comments based on the fix of https://github.com/apache/spark/pull/8742. Author: Yijie Shen <henry.yijieshen@gmail.com> Author: Yin Huai <yhuai@databricks.com> Closes #8823 from yhuai/fix_set_optimization.
* [SPARK-10540] Fixes flaky all-data-type testCheng Lian2015-09-181-66/+43
| | | | | | | | | | | | This PR breaks the original test case into multiple ones (one test case for each data type). In this way, test failure output can be much more readable. Within each test case, we build a table with two columns, one of them is for the data type to test, the other is an "index" column, which is used to sort the DataFrame and workaround [SPARK-10591] [1] [1]: https://issues.apache.org/jira/browse/SPARK-10591 Author: Cheng Lian <lian@databricks.com> Closes #8768 from liancheng/spark-10540/test-all-data-types.
* [SPARK-10451] [SQL] Prevent unnecessary serializations in ↵Yash Datta2015-09-181-14/+21
| | | | | | | | | | | | InMemoryColumnarTableScan Many of the fields in InMemoryColumnar scan and InMemoryRelation can be made transient. This reduces my 1000ms job to abt 700 ms . The task size reduces from 2.8 mb to ~1300kb Author: Yash Datta <Yash.Datta@guavus.com> Closes #8604 from saucam/serde.
* [SPARK-10684] [SQL] StructType.interpretedOrdering need not to be serializednavis.ryu2015-09-181-1/+3
| | | | | | | | | | | | | | | | | | | | | | | | Kryo fails with buffer overflow even with max value (2G). {noformat} org.apache.spark.SparkException: Kryo serialization failed: Buffer overflow. Available: 0, required: 1 Serialization trace: containsChild (org.apache.spark.sql.catalyst.expressions.BoundReference) child (org.apache.spark.sql.catalyst.expressions.SortOrder) array (scala.collection.mutable.ArraySeq) ordering (org.apache.spark.sql.catalyst.expressions.InterpretedOrdering) interpretedOrdering (org.apache.spark.sql.types.StructType) schema (org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema). To avoid this, increase spark.kryoserializer.buffer.max value. at org.apache.spark.serializer.KryoSerializerInstance.serialize(KryoSerializer.scala:263) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:240) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {noformat} Author: navis.ryu <navis@apache.org> Closes #8808 from navis/SPARK-10684.
* [SPARK-9522] [SQL] SparkSubmit process can not exit if kill application when ↵linweizhong2015-09-171-0/+6
| | | | | | | | | | HiveThriftServer was starting When we start HiveThriftServer, we will start SparkContext first, then start HiveServer2, if we kill application while HiveServer2 is starting then SparkContext will stop successfully, but SparkSubmit process can not exit. Author: linweizhong <linweizhong@huawei.com> Closes #7853 from Sephiroth-Lin/SPARK-9522.
* [SPARK-10639] [SQL] Need to convert UDAF's result from scala to sql typeYin Huai2015-09-176-12/+188
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-10639 Author: Yin Huai <yhuai@databricks.com> Closes #8788 from yhuai/udafConversion.
* [SPARK-10459] [SQL] Do not need to have ConvertToSafe for PythonUDFLiang-Chi Hsieh2015-09-171-0/+4
| | | | | | | | | | | | JIRA: https://issues.apache.org/jira/browse/SPARK-10459 As mentioned in the JIRA, `PythonUDF` actually could process `UnsafeRow`. Specially, the rows in `childResults` in `BatchPythonEvaluation` will be projected to a `MutableRow`. So I think we can enable `canProcessUnsafeRows` for `BatchPythonEvaluation` and get rid of redundant `ConvertToSafe`. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #8616 from viirya/pyudf-unsafe.
* Tiny style fix for d39f15ea2b8bed5342d2f8e3c1936f915c470783.Reynold Xin2015-09-161-1/+1
|
* [SPARK-9794] [SQL] Fix datetime parsing in SparkSQL.Kevin Cox2015-09-162-17/+42
| | | | | | | | | | This fixes https://issues.apache.org/jira/browse/SPARK-9794 by using a real ISO8601 parser. (courtesy of the xml component of the standard java library) cc: angelini Author: Kevin Cox <kevincox@kevincox.ca> Closes #8396 from kevincox/kevincox-sql-time-parsing.
* [SPARK-10050] [SPARKR] Support collecting data of MapType in DataFrame.Sun Rui2015-09-161-0/+6
| | | | | | | | | 1. Support collecting data of MapType from DataFrame. 2. Support data of MapType in createDataFrame. Author: Sun Rui <rui.sun@intel.com> Closes #8711 from sun-rui/SPARK-10050.
* [SPARK-9078] [SQL] Allow jdbc dialects to override the query used to check ↵sureshthalamati2015-09-154-4/+41
| | | | | | | | | | | | the table. Current implementation uses query with a LIMIT clause to find if table already exists. This syntax works only in some database systems. This patch changes the default query to the one that is likely to work on most databases, and adds a new method to the JdbcDialect abstract class to allow dialects to override the default query. I looked at using the JDBC meta data calls, it turns out there is no common way to find the current schema, catalog..etc. There is a new method Connection.getSchema() , but that is available only starting jdk1.7 , and existing jdbc drivers may not have implemented it. Other option was to use jdbc escape syntax clause for LIMIT, not sure on how well this supported in all the databases also. After looking at all the jdbc metadata options my conclusion was most common way is to use the simple select query with 'where 1 =0' , and allow dialects to customize as needed Author: sureshthalamati <suresh.thalamati@gmail.com> Closes #8676 from sureshthalamati/table_exists_spark-9078.
* [SPARK-10613] [SPARK-10624] [SQL] Reduce LocalNode tests dependency on ↵Andrew Or2015-09-1517-636/+468
| | | | | | | | | | | | SQLContext Instead of relying on `DataFrames` to verify our answers, we can just use simple arrays. This significantly simplifies the test logic for `LocalNode`s and reduces a lot of code duplicated from `SparkPlanTest`. This also fixes an additional issue [SPARK-10624](https://issues.apache.org/jira/browse/SPARK-10624) where the output of `TakeOrderedAndProjectNode` is not actually ordered. Author: Andrew Or <andrew@databricks.com> Closes #8764 from andrewor14/sql-local-tests-cleanup.
* [SPARK-10381] Fix mixup of taskAttemptNumber & attemptId in ↵Josh Rosen2015-09-153-5/+4
| | | | | | | | | | | | | | OutputCommitCoordinator When speculative execution is enabled, consider a scenario where the authorized committer of a particular output partition fails during the OutputCommitter.commitTask() call. In this case, the OutputCommitCoordinator is supposed to release that committer's exclusive lock on committing once that task fails. However, due to a unit mismatch (we used task attempt number in one place and task attempt id in another) the lock will not be released, causing Spark to go into an infinite retry loop. This bug was masked by the fact that the OutputCommitCoordinator does not have enough end-to-end tests (the current tests use many mocks). Other factors contributing to this bug are the fact that we have many similarly-named identifiers that have different semantics but the same data types (e.g. attemptNumber and taskAttemptId, with inconsistent variable naming which makes them difficult to distinguish). This patch adds a regression test and fixes this bug by always using task attempt numbers throughout this code. Author: Josh Rosen <joshrosen@databricks.com> Closes #8544 from JoshRosen/SPARK-10381.
* [SPARK-10612] [SQL] Add prepare to LocalNode.Reynold Xin2015-09-151-0/+8
| | | | | | | | The idea is that we should separate the function call that does memory reservation (i.e. prepare) from the function call that consumes the input (e.g. open()), so all operators can be a chance to reserve memory before they are all consumed. Author: Reynold Xin <rxin@databricks.com> Closes #8761 from rxin/SPARK-10612.
* [SPARK-10548] [SPARK-10563] [SQL] Fix concurrent SQL executionsAndrew Or2015-09-151-0/+101
| | | | | | | | | | | | | | | | *Note: this is for master branch only.* The fix for branch-1.5 is at #8721. The query execution ID is currently passed from a thread to its children, which is not the intended behavior. This led to `IllegalArgumentException: spark.sql.execution.id is already set` when running queries in parallel, e.g.: ``` (1 to 100).par.foreach { _ => sc.parallelize(1 to 5).map { i => (i, i) }.toDF("a", "b").count() } ``` The cause is `SparkContext`'s local properties are inherited by default. This patch adds a way to exclude keys we don't want to be inherited, and makes SQL go through that code path. Author: Andrew Or <andrew@databricks.com> Closes #8710 from andrewor14/concurrent-sql-executions.
* [SPARK-10475] [SQL] improve column prunning for Project on SortWenchen Fan2015-09-152-4/+26
| | | | | | | | Sometimes we can't push down the whole `Project` though `Sort`, but we still have a chance to push down part of it. Author: Wenchen Fan <cloud0fan@outlook.com> Closes #8644 from cloud-fan/column-prune.
* [SPARK-10437] [SQL] Support aggregation expressions in Order ByLiang-Chi Hsieh2015-09-152-4/+30
| | | | | | | | | | JIRA: https://issues.apache.org/jira/browse/SPARK-10437 If an expression in `SortOrder` is a resolved one, such as `count(1)`, the corresponding rule in `Analyzer` to make it work in order by will not be applied. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #8599 from viirya/orderby-agg.
* Revert "[SPARK-10300] [BUILD] [TESTS] Add support for test tags in ↵Marcelo Vanzin2015-09-155-30/+10
| | | | | | run-tests.py." This reverts commit 8abef21dac1a6538c4e4e0140323b83d804d602b.
* [SPARK-10300] [BUILD] [TESTS] Add support for test tags in run-tests.py.Marcelo Vanzin2015-09-155-10/+30
| | | | | | | | | | | | | | | This change does two things: - tag a few tests and adds the mechanism in the build to be able to disable those tags, both in maven and sbt, for both junit and scalatest suites. - add some logic to run-tests.py to disable some tags depending on what files have changed; that's used to disable expensive tests when a module hasn't explicitly been changed, to speed up testing for changes that don't directly affect those modules. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #8437 from vanzin/test-tags.
* Update version to 1.6.0-SNAPSHOT.Reynold Xin2015-09-154-4/+4
| | | | | | Author: Reynold Xin <rxin@databricks.com> Closes #8350 from rxin/1.6.
* [SPARK-10576] [BUILD] Move .java files out of src/main/scalaSean Owen2015-09-141-0/+0
| | | | | | | | Move .java files in `src/main/scala` to `src/main/java` root, except for `package-info.java` (to stay next to package.scala) Author: Sean Owen <sowen@cloudera.com> Closes #8736 from srowen/SPARK-10576.
* [SPARK-9996] [SPARK-9997] [SQL] Add local expand and NestedLoopJoin operatorszsxwing2015-09-147-15/+574
| | | | | | | | This PR is in conflict with #8535 and #8573. Will update this one when they are merged. Author: zsxwing <zsxwing@gmail.com> Closes #8642 from zsxwing/expand-nest-join.
* [SPARK-6981] [SQL] Factor out SparkPlanner and QueryExecution from SQLContextEdoardo Vacchi2015-09-146-128/+195
| | | | | | | | | | Alternative to PR #6122; in this case the refactored out classes are replaced by inner classes with the same name for backwards binary compatibility * process in a lighter-weight, backwards-compatible way Author: Edoardo Vacchi <uncommonnonsense@gmail.com> Closes #6356 from evacchi/sqlctx-refactoring-lite.
* [SPARK-10522] [SQL] Nanoseconds of Timestamp in Parquet should be positiveDavies Liu2015-09-142-14/+15
| | | | | | | | | | Or Hive can't read it back correctly. Thanks vanzin for report this. Author: Davies Liu <davies@databricks.com> Closes #8674 from davies/positive_nano.
* [SPARK-10584] [DOC] [SQL] Documentation about ↵Kousuke Saruta2015-09-141-4/+7
| | | | | | | | | | | spark.sql.hive.metastore.version is wrong. The default value of hive metastore version is 1.2.1 but the documentation says the value of `spark.sql.hive.metastore.version` is 0.13.1. Also, we cannot get the default value by `sqlContext.getConf("spark.sql.hive.metastore.version")`. Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #8739 from sarutak/SPARK-10584.
* [SPARK-9899] [SQL] log warning for direct output committer with speculation ↵Wenchen Fan2015-09-142-3/+15
| | | | | | | | | | | | | | | | enabled This is a follow-up of https://github.com/apache/spark/pull/8317. When speculation is enabled, there may be multiply tasks writing to the same path. Generally it's OK as we will write to a temporary directory first and only one task can commit the temporary directory to target path. However, when we use direct output committer, tasks will write data to target path directly without temporary directory. This causes problems like corrupted data. Please see [PR comment](https://github.com/apache/spark/pull/8191#issuecomment-131598385) for more details. Unfortunately, we don't have a simple flag to tell if a output committer will write to temporary directory or not, so for safety, we have to disable any customized output committer when `speculation` is true. Author: Wenchen Fan <cloud0fan@outlook.com> Closes #8687 from cloud-fan/direct-committer.