aboutsummaryrefslogtreecommitdiff
path: root/sql/core
Commit message (Collapse)AuthorAgeFilesLines
...
* [SPARK-16947][SQL] Support type coercion and foldable expression for inline ↵petermaxlee2016-08-192-0/+193
| | | | | | | | | | | | | | | | | | | tables ## What changes were proposed in this pull request? This patch improves inline table support with the following: 1. Support type coercion. 2. Support using foldable expressions. Previously only literals were supported. 3. Improve error message handling. 4. Improve test coverage. ## How was this patch tested? Added a new unit test suite ResolveInlineTablesSuite and a new file-based end-to-end test inline-table.sql. Author: petermaxlee <petermaxlee@gmail.com> Closes #14676 from petermaxlee/SPARK-16947.
* [SPARK-17117][SQL] 1 / NULL should not fail analysispetermaxlee2016-08-182-20/+76
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch fixes the problem described in SPARK-17117, i.e. "SELECT 1 / NULL" throws an analysis exception: ``` org.apache.spark.sql.AnalysisException: cannot resolve '(1 / NULL)' due to data type mismatch: differing types in '(1 / NULL)' (int and null). ``` The problem is that division type coercion did not take null type into account. ## How was this patch tested? A unit test for the type coercion, and a few end-to-end test cases using SQLQueryTestSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #14695 from petermaxlee/SPARK-17117.
* [SPARK-17069] Expose spark.range() as table-valued function in SQLEric Liang2016-08-182-0/+107
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This adds analyzer rules for resolving table-valued functions, and adds one builtin implementation for range(). The arguments for range() are the same as those of `spark.range()`. ## How was this patch tested? Unit tests. cc hvanhovell Author: Eric Liang <ekl@databricks.com> Closes #14656 from ericl/sc-4309.
* [SPARK-16391][SQL] Support partial aggregation for reduceGroupsReynold Xin2016-08-183-5/+146
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch introduces a new private ReduceAggregator interface that is a subclass of Aggregator. ReduceAggregator only requires a single associative and commutative reduce function. ReduceAggregator is also used to implement KeyValueGroupedDataset.reduceGroups in order to support partial aggregation. Note that the pull request was initially done by viirya. ## How was this patch tested? Covered by original tests for reduceGroups, as well as a new test suite for ReduceAggregator. Author: Reynold Xin <rxin@databricks.com> Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14576 from rxin/reduceAggregator.
* [SPARK-16995][SQL] TreeNodeException when flat mapping ↵Liang-Chi Hsieh2016-08-181-0/+13
| | | | | | | | | | | | | | | | | | | | | | | | | | | RelationalGroupedDataset created from DataFrame containing a column created with lit/expr ## What changes were proposed in this pull request? A TreeNodeException is thrown when executing the following minimal example in Spark 2.0. import spark.implicits._ case class test (x: Int, q: Int) val d = Seq(1).toDF("x") d.withColumn("q", lit(0)).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show d.withColumn("q", expr("0")).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show The problem is at `FoldablePropagation`. The rule will do `transformExpressions` on `LogicalPlan`. The query above contains a `MapGroups` which has a parameter `dataAttributes:Seq[Attribute]`. One attributes in `dataAttributes` will be transformed to an `Alias(literal(0), _)` in `FoldablePropagation`. `Alias` is not an `Attribute` and causes the error. We can't easily detect such type inconsistency during transforming expressions. A direct approach to this problem is to skip doing `FoldablePropagation` on object operators as they should not contain such expressions. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14648 from viirya/flat-mapping.
* [SPARK-17096][SQL][STREAMING] Improve exception string reported through the ↵Tathagata Das2016-08-174-14/+10
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | StreamingQueryListener ## What changes were proposed in this pull request? Currently, the stackTrace (as `Array[StackTraceElements]`) reported through StreamingQueryListener.onQueryTerminated is useless as it has the stack trace of where StreamingQueryException is defined, not the stack trace of underlying exception. For example, if a streaming query fails because of a / by zero exception in a task, the `QueryTerminated.stackTrace` will have ``` org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:211) org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124) ``` This is basically useless, as it is location where the StreamingQueryException was defined. What we want is Here is the right way to reason about what should be posted as through StreamingQueryListener.onQueryTerminated - The actual exception could either be a SparkException, or an arbitrary exception. - SparkException reports the relevant executor stack trace of a failed task as a string in the the exception message. The `Array[StackTraceElements]` returned by `SparkException.stackTrace()` is mostly irrelevant. - For any arbitrary exception, the `Array[StackTraceElements]` returned by `exception.stackTrace()` may be relevant. - When there is an error in a streaming query, it's hard to reason whether the `Array[StackTraceElements]` is useful or not. In fact, it is not clear whether it is even useful to report the stack trace as this array of Java objects. It may be sufficient to report the strack trace as a string, along with the message. This is how Spark reported executor stra - Hence, this PR simplifies the API by removing the array `stackTrace` from `QueryTerminated`. Instead the `exception` returns a string containing the message and the stack trace of the actual underlying exception that failed the streaming query (i.e. not that of the StreamingQueryException). If anyone is interested in the actual stack trace as an array, can always access them through `streamingQuery.exception` which returns the exception object. With this change, if a streaming query fails because of a / by zero exception in a task, the `QueryTerminated.exception` will be ``` org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent failure: Lost task 1.0 in stage 0.0 (TID 1, localhost): java.lang.ArithmeticException: / by zero at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply$mcII$sp(StreamingQueryListenerSuite.scala:153) at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply(StreamingQueryListenerSuite.scala:153) at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply(StreamingQueryListenerSuite.scala:153) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370) at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:232) at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:226) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70) at org.apache.spark.scheduler.Task.run(Task.scala:86) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) 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:744) Driver stacktrace: at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1429) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1417) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1416) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1416) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802) ... ``` It contains the relevant executor stack trace. In a case non-SparkException, if the streaming source MemoryStream throws an exception, exception message will have the relevant stack trace. ``` java.lang.RuntimeException: this is the exception message at org.apache.spark.sql.execution.streaming.MemoryStream.getBatch(memory.scala:103) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$5.apply(StreamExecution.scala:316) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$5.apply(StreamExecution.scala:313) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) at scala.collection.Iterator$class.foreach(Iterator.scala:893) at scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25) at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241) at org.apache.spark.sql.execution.streaming.StreamProgress.flatMap(StreamProgress.scala:25) at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatch(StreamExecution.scala:313) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:197) at org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43) at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:187) at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124) ``` Note that this change in the public `QueryTerminated` class is okay as the APIs are still experimental. ## How was this patch tested? Unit tests that test whether the right information is present in the exception message reported through QueryTerminated object. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #14675 from tdas/SPARK-17096.
* [SPARK-16736][CORE][SQL] purge superfluous fs callsSteve Loughran2016-08-171-15/+17
| | | | | | | | | | | | | | | A review of the code, working back from Hadoop's `FileSystem.exists()` and `FileSystem.isDirectory()` code, then removing uses of the calls when superfluous. 1. delete is harmless if called on a nonexistent path, so don't do any checks before deletes 1. any `FileSystem.exists()` check before `getFileStatus()` or `open()` is superfluous as the operation itself does the check. Instead the `FileNotFoundException` is caught and triggers the downgraded path. When a `FileNotFoundException` was thrown before, the code still creates a new FNFE with the error messages. Though now the inner exceptions are nested, for easier diagnostics. Initially, relying on Jenkins test runs. One troublespot here is that some of the codepaths are clearly error situations; it's not clear that they have coverage anyway. Trying to create the failure conditions in tests would be ideal, but it will also be hard. Author: Steve Loughran <stevel@apache.org> Closes #14371 from steveloughran/cloud/SPARK-16736-superfluous-fs-calls.
* [SPARK-17102][SQL] bypass UserDefinedGenerator for json format checkWenchen Fan2016-08-171-1/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? We use reflection to convert `TreeNode` to json string, and currently don't support arbitrary object. `UserDefinedGenerator` takes a function object, so we should skip json format test for it, or the tests can be flacky, e.g. `DataFrameSuite.simple explode`, this test always fail with scala 2.10(branch 1.6 builds with scala 2.10 by default), but pass with scala 2.11(master branch builds with scala 2.11 by default). ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #14679 from cloud-fan/json.
* [SPARK-17106] [SQL] Simplify the SubqueryExpression interfaceHerman van Hovell2016-08-174-36/+20
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The current subquery expression interface contains a little bit of technical debt in the form of a few different access paths to get and set the query contained by the expression. This is confusing to anyone who goes over this code. This PR unifies these access paths. ## How was this patch tested? (Existing tests) Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14685 from hvanhovell/SPARK-17106.
* [SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows ↵Kazuaki Ishizaki2016-08-171-0/+40
| | | | | | | | | | | | | | | | | | beyond 64 KB ## What changes were proposed in this pull request? This PR splits the generated code for ```SafeProjection.apply``` by using ```ctx.splitExpressions()```. This is because the large code body for ```NewInstance``` may grow beyond 64KB bytecode size for ```apply()``` method. Here is [the original PR](https://github.com/apache/spark/pull/13243) for SPARK-15285. However, it breaks a build with Scala 2.10 since Scala 2.10 does not a case class with large number of members. Thus, it was reverted by [this commit](https://github.com/apache/spark/commit/fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf). ## How was this patch tested? Added new tests by using `DefinedByConstructorParams` instead of case class for scala-2.10 Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #14670 from kiszk/SPARK-15285-2.
* [SPARK-17068][SQL] Make view-usage visible during analysisHerman van Hovell2016-08-163-5/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds a field to subquery alias in order to make the usage of views in a resolved `LogicalPlan` more visible (and more understandable). For example, the following view and query: ```sql create view constants as select 1 as id union all select 1 union all select 42 select * from constants; ``` ...now yields the following analyzed plan: ``` Project [id#39] +- SubqueryAlias c, `default`.`constants` +- Project [gen_attr_0#36 AS id#39] +- SubqueryAlias gen_subquery_0 +- Union :- Union : :- Project [1 AS gen_attr_0#36] : : +- OneRowRelation$ : +- Project [1 AS gen_attr_1#37] : +- OneRowRelation$ +- Project [42 AS gen_attr_2#38] +- OneRowRelation$ ``` ## How was this patch tested? Added tests for the two code paths in `SessionCatalogSuite` (sql/core) and `HiveMetastoreCatalogSuite` (sql/hive) Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14657 from hvanhovell/SPARK-17068.
* [SPARK-17084][SQL] Rename ParserUtils.assert to validateHerman van Hovell2016-08-161-3/+2
| | | | | | | | | | | | ## What changes were proposed in this pull request? This PR renames `ParserUtils.assert` to `ParserUtils.validate`. This is done because this method is used to check requirements, and not to check if the program is in an invalid state. ## How was this patch tested? Simple rename. Compilation should do. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14665 from hvanhovell/SPARK-17084.
* [SPARK-17034][SQL] adds expression UnresolvedOrdinal to represent the ↵Sean Zhong2016-08-162-6/+28
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ordinals in GROUP BY or ORDER BY ## What changes were proposed in this pull request? This PR adds expression `UnresolvedOrdinal` to represent the ordinal in GROUP BY or ORDER BY, and fixes the rules when resolving ordinals. Ordinals in GROUP BY or ORDER BY like `1` in `order by 1` or `group by 1` should be considered as unresolved before analysis. But in current code, it uses `Literal` expression to store the ordinal. This is inappropriate as `Literal` itself is a resolved expression, it gives the user a wrong message that the ordinals has already been resolved. ### Before this change Ordinal is stored as `Literal` expression ``` scala> sc.setLogLevel("TRACE") scala> sql("select a from t group by 1 order by 1") ... 'Sort [1 ASC], true +- 'Aggregate [1], ['a] +- 'UnresolvedRelation `t ``` For query: ``` scala> Seq(1).toDF("a").createOrReplaceTempView("t") scala> sql("select count(a), a from t group by 2 having a > 0").show ``` During analysis, the intermediate plan before applying rule `ResolveAggregateFunctions` is: ``` 'Filter ('a > 0) +- Aggregate [2], [count(1) AS count(1)#83L, a#81] +- LocalRelation [value#7 AS a#9] ``` Before this PR, rule `ResolveAggregateFunctions` believes all expressions of `Aggregate` have already been resolved, and tries to resolve the expressions in `Filter` directly. But this is wrong, as ordinal `2` in Aggregate is not really resolved! ### After this change Ordinals are stored as `UnresolvedOrdinal`. ``` scala> sc.setLogLevel("TRACE") scala> sql("select a from t group by 1 order by 1") ... 'Sort [unresolvedordinal(1) ASC], true +- 'Aggregate [unresolvedordinal(1)], ['a] +- 'UnresolvedRelation `t` ``` ## How was this patch tested? Unit tests. Author: Sean Zhong <seanzhong@databricks.com> Closes #14616 from clockfly/spark-16955.
* [SPARK-16916][SQL] serde/storage properties should not have limitationsWenchen Fan2016-08-153-11/+0
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `CatalogStorageFormat.properties` can be used in 2 ways: 1. for hive tables, it stores the serde properties. 2. for data source tables, it stores the data source options, e.g. `path`, `skipHiveMetadata`, etc. however, both of them have nothing to do with data source properties, e.g. `spark.sql.sources.provider`, so they should not have limitations about data source properties. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #14506 from cloud-fan/table-prop.
* [SPARK-17065][SQL] Improve the error message when encountering an ↵Shixiong Zhu2016-08-151-39/+52
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | incompatible DataSourceRegister ## What changes were proposed in this pull request? Add an instruction to ask the user to remove or upgrade the incompatible DataSourceRegister in the error message. ## How was this patch tested? Test command: ``` build/sbt -Dscala-2.10 package SPARK_SCALA_VERSION=2.10 bin/spark-shell --packages ai.h2o:sparkling-water-core_2.10:1.6.5 scala> Seq(1).toDS().write.format("parquet").save("foo") ``` Before: ``` java.util.ServiceConfigurationError: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.h2o.DefaultSource could not be instantiated at java.util.ServiceLoader.fail(ServiceLoader.java:232) at java.util.ServiceLoader.access$100(ServiceLoader.java:185) at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384) at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404) at java.util.ServiceLoader$1.next(ServiceLoader.java:480) ... Caused by: java.lang.NoClassDefFoundError: org/apache/spark/Logging at java.lang.ClassLoader.defineClass1(Native Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:760) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) at java.net.URLClassLoader.defineClass(URLClassLoader.java:467) at java.net.URLClassLoader.access$100(URLClassLoader.java:73) at java.net.URLClassLoader$1.run(URLClassLoader.java:368) at java.net.URLClassLoader$1.run(URLClassLoader.java:362) at java.security.AccessController.doPrivileged(Native Method) ... ``` After: ``` java.lang.ClassNotFoundException: Detected an incompatible DataSourceRegister. Please remove the incompatible library from classpath or upgrade it. Error: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.h2o.DefaultSource could not be instantiated at org.apache.spark.sql.execution.datasources.DataSource.lookupDataSource(DataSource.scala:178) at org.apache.spark.sql.execution.datasources.DataSource.providingClass$lzycompute(DataSource.scala:79) at org.apache.spark.sql.execution.datasources.DataSource.providingClass(DataSource.scala:79) at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:441) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:213) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:196) ... ``` Author: Shixiong Zhu <shixiong@databricks.com> Closes #14651 from zsxwing/SPARK-17065.
* [SPARK-16671][CORE][SQL] Consolidate code to do variable substitution.Marcelo Vanzin2016-08-153-101/+18
| | | | | | | | | | | | | | | | Both core and sql have slightly different code that does variable substitution of config values. This change refactors that code and encapsulates the logic of reading config values and expading variables in a new helper class, which can be configured so that both core and sql can use it without losing existing functionality, and allows for easier testing and makes it easier to add more features in the future. Tested with existing and new unit tests, and by running spark-shell with some configs referencing variables and making sure it behaved as expected. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #14468 from vanzin/SPARK-16671.
* [SPARK-16966][SQL][CORE] App Name is a randomUUID even when "spark.app.name" ↵Sean Owen2016-08-132-4/+8
| | | | | | | | | | | | | | | | | | | exists ## What changes were proposed in this pull request? Don't override app name specified in `SparkConf` with a random app name. Only set it if the conf has no app name even after options have been applied. See also https://github.com/apache/spark/pull/14602 This is similar to Sherry302 's original proposal in https://github.com/apache/spark/pull/14556 ## How was this patch tested? Jenkins test, with new case reproducing the bug Author: Sean Owen <sowen@cloudera.com> Closes #14630 from srowen/SPARK-16966.2.
* [SPARK-16968] Add additional options in jdbc when creating a new tableGraceH2016-08-133-12/+51
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In the PR, we just allow the user to add additional options when create a new table in JDBC writer. The options can be table_options or partition_options. E.g., "CREATE TABLE t (name string) ENGINE=InnoDB DEFAULT CHARSET=utf8" Here is the usage example: ``` df.write.option("createTableOptions", "ENGINE=InnoDB DEFAULT CHARSET=utf8").jdbc(...) ``` ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) will apply test result soon. Author: GraceH <93113783@qq.com> Closes #14559 from GraceH/jdbc_options.
* [SPARK-16771][SQL] WITH clause should not fall into infinite loop.Dongjoon Hyun2016-08-122-0/+71
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR changes the CTE resolving rule to use only **forward-declared** tables in order to prevent infinite loops. More specifically, new logic is like the following. * Resolve CTEs in `WITH` clauses first before replacing the main SQL body. * When resolving CTEs, only forward-declared CTEs or base tables are referenced. - Self-referencing is not allowed any more. - Cross-referencing is not allowed any more. **Reported Error Scenarios** ```scala scala> sql("WITH t AS (SELECT 1 FROM t) SELECT * FROM t") java.lang.StackOverflowError ... scala> sql("WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1, t2") java.lang.StackOverflowError ... ``` Note that `t`, `t1`, and `t2` are not declared in database. Spark falls into infinite loops before resolving table names. ## How was this patch tested? Pass the Jenkins tests with new two testcases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14397 from dongjoon-hyun/SPARK-16771-TREENODE.
* [SPARK-17013][SQL] Parse negative numeric literalspetermaxlee2016-08-112-44/+26
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch updates the SQL parser to parse negative numeric literals as numeric literals, instead of unary minus of positive literals. This allows the parser to parse the minimal value for each data type, e.g. "-32768S". ## How was this patch tested? Updated test cases. Author: petermaxlee <petermaxlee@gmail.com> Closes #14608 from petermaxlee/SPARK-17013.
* [SPARK-16975][SQL] Column-partition path starting '_' should be handled ↵Dongjoon Hyun2016-08-125-4/+14
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | correctly ## What changes were proposed in this pull request? Currently, Spark ignores path names starting with underscore `_` and `.`. This causes read-failures for the column-partitioned file data sources whose partition column names starts from '_', e.g. `_col`. **Before** ```scala scala> spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet") scala> spark.read.parquet("/tmp/parquet") org.apache.spark.sql.AnalysisException: Unable to infer schema for ParquetFormat at /tmp/parquet20. It must be specified manually; ``` **After** ```scala scala> spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet") scala> spark.read.parquet("/tmp/parquet") res2: org.apache.spark.sql.DataFrame = [id: bigint, _locality_code: int] ``` ## How was this patch tested? Pass the Jenkins with a new test case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14585 from dongjoon-hyun/SPARK-16975-PARQUET.
* [SPARK-16434][SQL] Avoid per-record type dispatch in JSON when readinghyukjinkwon2016-08-125-216/+297
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, `JacksonParser.parse` is doing type-based dispatch for each row to convert the tokens to appropriate values for Spark. It might not have to be done like this because the schema is already kept. So, appropriate converters can be created first according to the schema once, and then apply them to each row. This PR corrects `JacksonParser` so that it creates all converters for the schema once and then applies them to each row rather than type dispatching for every row. Benchmark was proceeded with the codes below: #### Parser tests **Before** ```scala test("Benchmark for JSON converter") { val N = 500 << 8 val row = """{"struct":{"field1": true, "field2": 92233720368547758070}, "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, "arrayOfString":["str1", "str2"], "arrayOfInteger":[1, 2147483647, -2147483648], "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], "arrayOfBoolean":[true, false, true], "arrayOfNull":[null, null, null, null], "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] }""" val data = List.fill(N)(row) val dummyOption = new JSONOptions(Map.empty[String, String]) val schema = InferSchema.infer(spark.sparkContext.parallelize(Seq(row)), "", dummyOption) val factory = new JsonFactory() val benchmark = new Benchmark("JSON converter", N) benchmark.addCase("convert JSON file", 10) { _ => data.foreach { input => val parser = factory.createParser(input) parser.nextToken() JacksonParser.convertRootField(factory, parser, schema) } } benchmark.run() } ``` ``` JSON converter: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ convert JSON file 1697 / 1807 0.1 13256.9 1.0X ``` **After** ```scala test("Benchmark for JSON converter") { val N = 500 << 8 val row = """{"struct":{"field1": true, "field2": 92233720368547758070}, "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, "arrayOfString":["str1", "str2"], "arrayOfInteger":[1, 2147483647, -2147483648], "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], "arrayOfBoolean":[true, false, true], "arrayOfNull":[null, null, null, null], "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] }""" val data = List.fill(N)(row) val dummyOption = new JSONOptions(Map.empty[String, String], new SQLConf()) val schema = InferSchema.infer(spark.sparkContext.parallelize(Seq(row)), dummyOption) val benchmark = new Benchmark("JSON converter", N) benchmark.addCase("convert JSON file", 10) { _ => val parser = new JacksonParser(schema, dummyOption) data.foreach { input => parser.parse(input) } } benchmark.run() } ``` ``` JSON converter: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ convert JSON file 1401 / 1461 0.1 10947.4 1.0X ``` It seems parsing time is improved by roughly ~20% #### End-to-End test ```scala test("Benchmark for JSON reader") { val N = 500 << 8 val row = """{"struct":{"field1": true, "field2": 92233720368547758070}, "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, "arrayOfString":["str1", "str2"], "arrayOfInteger":[1, 2147483647, -2147483648], "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], "arrayOfBoolean":[true, false, true], "arrayOfNull":[null, null, null, null], "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] }""" val df = spark.sqlContext.read.json(spark.sparkContext.parallelize(List.fill(N)(row))) withTempPath { path => df.write.format("json").save(path.getCanonicalPath) val benchmark = new Benchmark("JSON reader", N) benchmark.addCase("reading JSON file", 10) { _ => spark.read.format("json").load(path.getCanonicalPath).collect() } benchmark.run() } } ``` **Before** ``` JSON reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading JSON file 6485 / 6924 0.0 50665.0 1.0X ``` **After** ``` JSON reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading JSON file 6350 / 6529 0.0 49609.3 1.0X ``` ## How was this patch tested? Existing test cases should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14102 from HyukjinKwon/SPARK-16434.
* [SPARK-17018][SQL] literals.sql for testing literal parsingpetermaxlee2016-08-115-62/+476
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch adds literals.sql for testing literal parsing end-to-end in SQL. ## How was this patch tested? The patch itself is only about adding test cases. Author: petermaxlee <petermaxlee@gmail.com> Closes #14598 from petermaxlee/SPARK-17018-2.
* [SPARK-17021][SQL] simplify the constructor parameters of QuantileSummariesWenchen Fan2016-08-111-10/+11
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1. `sampled` doesn't need to be `ArrayBuffer`, we never update it, but assign new value 2. `count` doesn't need to be `var`, we never mutate it. 3. `headSampled` doesn't need to be in constructor, we never pass a non-empty `headSampled` to constructor ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #14603 from cloud-fan/simply.
* [SPARK-16958] [SQL] Reuse subqueries within the same queryDavies Liu2016-08-115-47/+206
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? There could be multiple subqueries that generate same results, we could re-use the result instead of running it multiple times. This PR also cleanup up how we run subqueries. For SQL query ```sql select id,(select avg(id) from t) from t where id > (select avg(id) from t) ``` The explain is ``` == Physical Plan == *Project [id#15L, Subquery subquery29 AS scalarsubquery()#35] : +- Subquery subquery29 : +- *HashAggregate(keys=[], functions=[avg(id#15L)]) : +- Exchange SinglePartition : +- *HashAggregate(keys=[], functions=[partial_avg(id#15L)]) : +- *Range (0, 1000, splits=4) +- *Filter (cast(id#15L as double) > Subquery subquery29) : +- Subquery subquery29 : +- *HashAggregate(keys=[], functions=[avg(id#15L)]) : +- Exchange SinglePartition : +- *HashAggregate(keys=[], functions=[partial_avg(id#15L)]) : +- *Range (0, 1000, splits=4) +- *Range (0, 1000, splits=4) ``` The visualized plan: ![reuse-subquery](https://cloud.githubusercontent.com/assets/40902/17573229/e578d93c-5f0d-11e6-8a3c-0150d81d3aed.png) ## How was this patch tested? Existing tests. Author: Davies Liu <davies@databricks.com> Closes #14548 from davies/subq.
* [SPARK-17015][SQL] group-by/order-by ordinal and arithmetic testspetermaxlee2016-08-117-220/+601
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch adds three test files: 1. arithmetic.sql.out 2. order-by-ordinal.sql 3. group-by-ordinal.sql This includes https://github.com/apache/spark/pull/14594. ## How was this patch tested? This is a test case change. Author: petermaxlee <petermaxlee@gmail.com> Closes #14595 from petermaxlee/SPARK-17015.
* [SPARK-17011][SQL] Support testing exceptions in SQLQueryTestSuitepetermaxlee2016-08-109-68/+161
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch adds exception testing to SQLQueryTestSuite. When there is an exception in query execution, the query result contains the the exception class along with the exception message. As part of this, I moved some additional test cases for limit from SQLQuerySuite over to SQLQueryTestSuite. ## How was this patch tested? This is a test harness change. Author: petermaxlee <petermaxlee@gmail.com> Closes #14592 from petermaxlee/SPARK-17011.
* [SPARK-17007][SQL] Move test data files into a test-data folderpetermaxlee2016-08-1040-39/+40
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch moves all the test data files in sql/core/src/test/resources to sql/core/src/test/resources/test-data, so we don't clutter the top level sql/core/src/test/resources. Also deleted sql/core/src/test/resources/old-repeated.parquet since it is no longer used. The change will make it easier to spot sql-tests directory. ## How was this patch tested? This is a test-only change. Author: petermaxlee <petermaxlee@gmail.com> Closes #14589 from petermaxlee/SPARK-17007.
* [SPARK-17008][SPARK-17009][SQL] Normalization and isolation in ↵petermaxlee2016-08-108-65/+180
| | | | | | | | | | | | | | | | | | | SQLQueryTestSuite. ## What changes were proposed in this pull request? This patch enhances SQLQueryTestSuite in two ways: 1. SPARK-17009: Use a new SparkSession for each test case to provide stronger isolation (e.g. config changes in one test case does not impact another). That said, we do not currently isolate catalog changes. 2. SPARK-17008: Normalize query output using sorting, inspired by HiveComparisonTest. I also ported a few new test cases over from SQLQuerySuite. ## How was this patch tested? This is a test harness update. Author: petermaxlee <petermaxlee@gmail.com> Closes #14590 from petermaxlee/SPARK-17008.
* [SPARK-16928] [SQL] Recursive call of ColumnVector::getInt() breaks JIT inliningQifan Pu2016-08-104-25/+54
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In both `OnHeapColumnVector` and `OffHeapColumnVector`, we implemented `getInt()` with the following code pattern: ``` public int getInt(int rowId) { if (dictionary == null) { return intData[rowId]; } else { return dictionary.decodeToInt(dictionaryIds.getInt(rowId)); } } ``` As `dictionaryIds` is also a `ColumnVector`, this results in a recursive call of `getInt()` and breaks JIT inlining. As a result, `getInt()` will not get inlined. We fix this by adding a separate method `getDictId()` specific for `dictionaryIds` to use. ## How was this patch tested? We tested the difference with the following aggregate query on a TPCDS dataset (with scale factor = 5): ``` select max(ss_sold_date_sk) as max_ss_sold_date_sk, from store_sales ``` The query runtime is improved, from 202ms (before) to 159ms (after). Author: Qifan Pu <qifan.pu@gmail.com> Closes #14513 from ooq/SPARK-16928.
* [SPARK-15639] [SPARK-16321] [SQL] Push down filter at RowGroups level for ↵Liang-Chi Hsieh2016-08-103-147/+122
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | parquet reader ## What changes were proposed in this pull request? The base class `SpecificParquetRecordReaderBase` used for vectorized parquet reader will try to get pushed-down filters from the given configuration. This pushed-down filters are used for RowGroups-level filtering. However, we don't set up the filters to push down into the configuration. In other words, the filters are not actually pushed down to do RowGroups-level filtering. This patch is to fix this and tries to set up the filters for pushing down to configuration for the reader. The benchmark that excludes the time of writing Parquet file: test("Benchmark for Parquet") { val N = 500 << 12 withParquetTable((0 until N).map(i => (101, i)), "t") { val benchmark = new Benchmark("Parquet reader", N) benchmark.addCase("reading Parquet file", 10) { iter => sql("SELECT _1 FROM t where t._1 < 100").collect() } benchmark.run() } } `withParquetTable` in default will run tests for vectorized reader non-vectorized readers. I only let it run vectorized reader. When we set the block size of parquet as 1024 to have multiple row groups. The benchmark is: Before this patch: The retrieved row groups: 8063 Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic Intel(R) Core(TM) i7-5557U CPU 3.10GHz Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading Parquet file 825 / 1233 2.5 402.6 1.0X After this patch: The retrieved row groups: 0 Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic Intel(R) Core(TM) i7-5557U CPU 3.10GHz Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading Parquet file 306 / 503 6.7 149.6 1.0X Next, I run the benchmark for non-pushdown case using the same benchmark code but with disabled pushdown configuration. This time the parquet block size is default value. Before this patch: Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic Intel(R) Core(TM) i7-5557U CPU 3.10GHz Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading Parquet file 136 / 238 15.0 66.5 1.0X After this patch: Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic Intel(R) Core(TM) i7-5557U CPU 3.10GHz Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading Parquet file 124 / 193 16.5 60.7 1.0X For non-pushdown case, from the results, I think this patch doesn't affect normal code path. I've manually output the `totalRowCount` in `SpecificParquetRecordReaderBase` to see if this patch actually filter the row-groups. When running the above benchmark: After this patch: `totalRowCount = 0` Before this patch: `totalRowCount = 1024000` ## How was this patch tested? Existing tests should be passed. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #13701 from viirya/vectorized-reader-push-down-filter2.
* [SPARK-15899][SQL] Fix the construction of the file path with hadoop Pathavulanov2016-08-103-27/+35
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix the construction of the file path. Previous way of construction caused the creation of incorrect path on Windows. ## How was this patch tested? Run SQL unit tests on Windows Author: avulanov <nashb@yandex.ru> Closes #13868 from avulanov/SPARK-15899-file.
* [SPARK-16866][SQL] Infrastructure for file-based SQL end-to-end testspetermaxlee2016-08-105-36/+266
| | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch introduces SQLQueryTestSuite, a basic framework for end-to-end SQL test cases defined in spark/sql/core/src/test/resources/sql-tests. This is a more standard way to test SQL queries end-to-end in different open source database systems, because it is more manageable to work with files. This is inspired by HiveCompatibilitySuite, but simplified for general Spark SQL tests. Once this is merged, I can work towards porting SQLQuerySuite over, and eventually also move the existing HiveCompatibilitySuite to use this framework. Unlike HiveCompatibilitySuite, SQLQueryTestSuite compares both the output schema and the output data (in string form). When there is a mismatch, the error message looks like the following: ``` [info] - blacklist.sql !!! IGNORED !!! [info] - number-format.sql *** FAILED *** (2 seconds, 405 milliseconds) [info] Expected "...147483648 -214748364[8]", but got "...147483648 -214748364[9]" Result should match for query #1 (SQLQueryTestSuite.scala:171) [info] org.scalatest.exceptions.TestFailedException: [info] at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:495) [info] at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555) [info] at org.scalatest.Assertions$class.assertResult(Assertions.scala:1171) ``` ## How was this patch tested? This is a test infrastructure change. Author: petermaxlee <petermaxlee@gmail.com> Closes #14472 from petermaxlee/SPARK-16866.
* [SPARK-16324][SQL] regexp_extract should doc that it returns empty string ↵Sean Owen2016-08-102-1/+6
| | | | | | | | | | | | | | | | when match fails ## What changes were proposed in this pull request? Doc that regexp_extract returns empty string when regex or group does not match ## How was this patch tested? Jenkins test, with a few new test cases Author: Sean Owen <sowen@cloudera.com> Closes #14525 from srowen/SPARK-16324.
* Typo: Fow -> ForAndrew Ash2016-08-101-1/+1
| | | | | | Author: Andrew Ash <andrew@andrewash.com> Closes #14563 from ash211/patch-8.
* [SPARK-16185][SQL] Better Error Messages When Creating Table As Select ↵gatorsmile2016-08-103-1/+45
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Without Enabling Hive Support #### What changes were proposed in this pull request? When we do not turn on the Hive Support, the following query generates a confusing error message by Planner: ```Scala sql("CREATE TABLE t2 SELECT a, b from t1") ``` ``` assertion failed: No plan for CreateTable CatalogTable( Table: `t2` Created: Tue Aug 09 23:45:32 PDT 2016 Last Access: Wed Dec 31 15:59:59 PST 1969 Type: MANAGED Provider: hive Storage(InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat)), ErrorIfExists +- Relation[a#19L,b#20L] parquet java.lang.AssertionError: assertion failed: No plan for CreateTable CatalogTable( Table: `t2` Created: Tue Aug 09 23:45:32 PDT 2016 Last Access: Wed Dec 31 15:59:59 PST 1969 Type: MANAGED Provider: hive Storage(InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat)), ErrorIfExists +- Relation[a#19L,b#20L] parquet ``` This PR is to issue a better error message: ``` Hive support is required to use CREATE Hive TABLE AS SELECT ``` #### How was this patch tested? Added test cases in `DDLSuite.scala` Author: gatorsmile <gatorsmile@gmail.com> Closes #13886 from gatorsmile/createCatalogedTableAsSelect.
* [SPARK-10601][SQL] Support `MINUS` set operatorDongjoon Hyun2016-08-101-0/+10
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds `MINUS` set operator which is equivalent `EXCEPT DISTINCT`. This will slightly improve the compatibility with Oracle. ## How was this patch tested? Pass the Jenkins with newly added testcases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14570 from dongjoon-hyun/SPARK-10601.
* [SPARK-16905] SQL DDL: MSCK REPAIR TABLEDavies Liu2016-08-095-5/+199
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? MSCK REPAIR TABLE could be used to recover the partitions in external catalog based on partitions in file system. Another syntax is: ALTER TABLE table RECOVER PARTITIONS The implementation in this PR will only list partitions (not the files with a partition) in driver (in parallel if needed). ## How was this patch tested? Added unit tests for it and Hive compatibility test suite. Author: Davies Liu <davies@databricks.com> Closes #14500 from davies/repair_table.
* [SPARK-16964][SQL] Remove private[sql] and private[spark] from sql.execution ↵Reynold Xin2016-08-0962-176/+169
| | | | | | | | | | | | | | | | package ## What changes were proposed in this pull request? This package is meant to be internal, and as a result it does not make sense to mark things as private[sql] or private[spark]. It simply makes debugging harder when Spark developers need to inspect the plans at runtime. This patch removes all private[sql] and private[spark] visibility modifiers in org.apache.spark.sql.execution. ## How was this patch tested? N/A - just visibility changes. Author: Reynold Xin <rxin@databricks.com> Closes #14554 from rxin/remote-private.
* [SPARK-16940][SQL] `checkAnswer` should raise `TestFailedException` for ↵Dongjoon Hyun2016-08-091-1/+9
| | | | | | | | | | | | | | | | | | | | | wrong results ## What changes were proposed in this pull request? This PR fixes the following to make `checkAnswer` raise `TestFailedException` again instead of `java.util.NoSuchElementException: key not found: TZ` in the environments without `TZ` variable. Also, this PR adds `QueryTestSuite` class for testing `QueryTest` itself. ```scala - |Timezone Env: ${sys.env("TZ")} + |Timezone Env: ${sys.env.getOrElse("TZ", "")} ``` ## How was this patch tested? Pass the Jenkins tests with a new test suite. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14528 from dongjoon-hyun/SPARK-16940.
* [SPARK-16898][SQL] Adds argument type information for typed logical plan ↵Sean Zhong2016-08-092-6/+6
| | | | | | | | | | | | | | | | like MapElements, TypedFilter, and AppendColumn ## What changes were proposed in this pull request? This PR adds argument type information for typed logical plan like MapElements, TypedFilter, and AppendColumn, so that we can use these info in customized optimizer rule. ## How was this patch tested? Existing test. Author: Sean Zhong <seanzhong@databricks.com> Closes #14494 from clockfly/add_more_info_for_typed_operator.
* [SPARK-16749][SQL] Simplify processing logic in LEAD/LAG processing.Herman van Hovell2016-08-081-35/+18
| | | | | | | | | | | | ## What changes were proposed in this pull request? The logic for LEAD/LAG processing is more complex that it needs to be. This PR fixes that. ## How was this patch tested? Existing tests. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14376 from hvanhovell/SPARK-16749.
* [SPARK-16779][TRIVIAL] Avoid using postfix operators where they do not add ↵Holden Karau2016-08-085-9/+0
| | | | | | | | | | | | | | | | much and remove whitelisting ## What changes were proposed in this pull request? Avoid using postfix operation for command execution in SQLQuerySuite where it wasn't whitelisted and audit existing whitelistings removing postfix operators from most places. Some notable places where postfix operation remains is in the XML parsing & time units (seconds, millis, etc.) where it arguably can improve readability. ## How was this patch tested? Existing tests. Author: Holden Karau <holden@us.ibm.com> Closes #14407 from holdenk/SPARK-16779.
* [SPARK-16936][SQL] Case Sensitivity Support for Refresh Temp Tablegatorsmile2016-08-082-1/+26
| | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? Currently, the `refreshTable` API is always case sensitive. When users use the view name without the exact case match, the API silently ignores the call. Users might expect the command has been successfully completed. However, when users run the subsequent SQL commands, they might still get the exception, like ``` Job aborted due to stage failure: Task 1 in stage 4.0 failed 1 times, most recent failure: Lost task 1.0 in stage 4.0 (TID 7, localhost): java.io.FileNotFoundException: File file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-bd4b9ea6-9aec-49c5-8f05-01cff426211e/part-r-00000-0c84b915-c032-4f2e-abf5-1d48fdbddf38.snappy.parquet does not exist ``` This PR is to fix the issue. ### How was this patch tested? Added a test case. Author: gatorsmile <gatorsmile@gmail.com> Closes #14523 from gatorsmile/refreshTempTable.
* [SPARK-16457][SQL] Fix Wrong Messages when CTAS with a Partition By Clausegatorsmile2016-08-081-6/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? When doing a CTAS with a Partition By clause, we got a wrong error message. For example, ```SQL CREATE TABLE gen__tmp PARTITIONED BY (key string) AS SELECT key, value FROM mytable1 ``` The error message we get now is like ``` Operation not allowed: Schema may not be specified in a Create Table As Select (CTAS) statement(line 2, pos 0) ``` However, based on the code, the message we should get is like ``` Operation not allowed: A Create Table As Select (CTAS) statement is not allowed to create a partitioned table using Hive's file formats. Please use the syntax of "CREATE TABLE tableName USING dataSource OPTIONS (...) PARTITIONED BY ...\" to create a partitioned table through a CTAS statement.(line 2, pos 0) ``` Currently, partitioning columns is part of the schema. This PR fixes the bug by changing the detection orders. #### How was this patch tested? Added test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #14113 from gatorsmile/ctas.
* [SPARK-16906][SQL] Adds auxiliary info like input class and input schema in ↵Sean Zhong2016-08-085-7/+14
| | | | | | | | | | | | | | | | TypedAggregateExpression ## What changes were proposed in this pull request? This PR adds auxiliary info like input class and input schema in TypedAggregateExpression ## How was this patch tested? Manual test. Author: Sean Zhong <seanzhong@databricks.com> Closes #14501 from clockfly/typed_aggregation.
* [SPARK-16804][SQL] Correlated subqueries containing non-deterministic ↵Nattavut Sutyanyong2016-08-081-0/+29
| | | | | | | | | | | | | | | | | operations return incorrect results ## What changes were proposed in this pull request? This patch fixes the incorrect results in the rule ResolveSubquery in Catalyst's Analysis phase by returning an error message when the LIMIT is found in the path from the parent table to the correlated predicate in the subquery. ## How was this patch tested? ./dev/run-tests a new unit test on the problematic pattern. Author: Nattavut Sutyanyong <nsy.can@gmail.com> Closes #14411 from nsyca/master.
* [SPARK-16939][SQL] Fix build error by using `Tuple1` explicitly in ↵Dongjoon Hyun2016-08-071-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | StringFunctionsSuite ## What changes were proposed in this pull request? This PR aims to fix a build error on branch 1.6 at https://github.com/apache/spark/commit/8d8725208771a8815a60160a5a30dc6ea87a7e6a, but I think we had better have this consistently in master branch, too. It's because there exist other ongoing PR (https://github.com/apache/spark/pull/14525) about this. https://amplab.cs.berkeley.edu/jenkins/job/spark-branch-1.6-compile-maven-with-yarn-2.3/286/console ```scala [error] /home/jenkins/workspace/spark-branch-1.6-compile-maven-with-yarn-2.3/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala:82: value toDF is not a member of Seq[String] [error] val df = Seq("aaaac").toDF("s") [error] ^ ``` ## How was this patch tested? After passing Jenkins, run compilation test on branch 1.6. ``` build/mvn -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install ``` Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14526 from dongjoon-hyun/SPARK-16939.
* [SPARK-16409][SQL] regexp_extract with optional groups causes NPESean Owen2016-08-071-0/+8
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? regexp_extract actually returns null when it shouldn't when a regex matches but the requested optional group did not. This makes it return an empty string, as apparently designed. ## How was this patch tested? Additional unit test Author: Sean Owen <sowen@cloudera.com> Closes #14504 from srowen/SPARK-16409.
* [SPARK-16847][SQL] Prevent to potentially read corrupt statstics on binary ↵hyukjinkwon2016-08-061-2/+4
| | | | | | | | | | | | | | | | | | | | | | in Parquet vectorized reader ## What changes were proposed in this pull request? This problem was found in [PARQUET-251](https://issues.apache.org/jira/browse/PARQUET-251) and we disabled filter pushdown on binary columns in Spark before. We enabled this after upgrading Parquet but it seems there is potential incompatibility for Parquet files written in lower Spark versions. Currently, this does not happen in normal Parquet reader. However, In Spark, we implemented a vectorized reader, separately with Parquet's standard API. For normal Parquet reader this is being handled but not in the vectorized reader. It is okay to just pass `FileMetaData`. This is being handled in parquet-mr (See https://github.com/apache/parquet-mr/commit/e3b95020f777eb5e0651977f654c1662e3ea1f29). This will prevent loading corrupt statistics in each page in Parquet. This PR replaces the deprecated usage of constructor. ## How was this patch tested? N/A Author: hyukjinkwon <gurwls223@gmail.com> Closes #14450 from HyukjinKwon/SPARK-16847.