Commit graph

7080 commits

Author SHA1 Message Date
Ali Afroozeh 0a6043f683 [SPARK-32755][SQL] Maintain the order of expressions in AttributeSet and ExpressionSet
### What changes were proposed in this pull request?
This PR changes `AttributeSet` and `ExpressionSet` to maintain the insertion order of the elements. More specifically, we:
- change the underlying data structure of `AttributeSet` from `HashSet` to `LinkedHashSet` to maintain the insertion order.
- `ExpressionSet` already uses a list to keep track of the expressions, however, since it is extending Scala's immutable.Set class, operations such as map and flatMap are delegated to the immutable.Set itself. This means that the result of these operations is not an instance of ExpressionSet anymore, rather it's a implementation picked up by the parent class. We also remove this inheritance from `immutable.Set `and implement the needed methods directly. ExpressionSet has a very specific semantics and it does not make sense to extend `immutable.Set` anyway.
- change the `PlanStabilitySuite` to not sort the attributes, to be able to catch changes in the order of expressions in different runs.

### Why are the changes needed?
Expressions identity is based on the `ExprId` which is an auto-incremented number. This means that the same query can yield a query plan with different expression ids in different runs. `AttributeSet` and `ExpressionSet` internally use a `HashSet` as the underlying data structure, and therefore cannot guarantee the a fixed order of operations in different runs. This can be problematic in cases we like to check for plan changes in different runs.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Passes `PlanStabilitySuite` after regenerating the golden files.

Closes #29598 from dbaliafroozeh/FixOrderOfExpressions.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2020-09-03 13:56:03 +02:00
Yuanjian Li 95f1e9549b [SPARK-32782][SS] Refactor StreamingRelationV2 and move it to catalyst
### What changes were proposed in this pull request?
Move StreamingRelationV2 to the catalyst module and bind with the Table interface.

### Why are the changes needed?
Currently, the StreamingRelationV2 is bind with TableProvider. Since the V2 relation is not bound with `DataSource`, to make it more flexible and have better expansibility, it should be moved to the catalyst module and bound with the Table interface. We did a similar thing for DataSourceV2Relation.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing UT.

Closes #29633 from xuanyuanking/SPARK-32782.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-03 16:04:36 +09:00
Kent Yao 1fba286407 [SPARK-32781][SQL] Non-ASCII characters are mistakenly omitted in the middle of intervals
### What changes were proposed in this pull request?

This PR fails the interval values parsing when they contain non-ASCII characters which are silently omitted right now.

e.g. the case below should be invalid

```
select interval 'interval中文 1 day'
```

### Why are the changes needed?

bugfix, intervals should fail when containing invalid characters

### Does this PR introduce _any_ user-facing change?

yes,

#### before

select interval 'interval中文 1 day'  results 1 day, now it fails with

```
org.apache.spark.sql.catalyst.parser.ParseException

Cannot parse the INTERVAL value: interval中文 1 day
```

### How was this patch tested?

new tests

Closes #29632 from yaooqinn/SPARK-32781.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-03 04:56:40 +00:00
angerszhu 5e6173ebef [SPARK-31670][SQL] Trim unnecessary Struct field alias in Aggregate/GroupingSets
### What changes were proposed in this pull request?
Struct field both in GROUP BY and Aggregate Expresison with CUBE/ROLLUP/GROUPING SET will failed when analysis.

```
test("SPARK-31670") {
  withTable("t1") {
      sql(
        """
          |CREATE TEMPORARY VIEW t(a, b, c) AS
          |SELECT * FROM VALUES
          |('A', 1, NAMED_STRUCT('row_id', 1, 'json_string', '{"i": 1}')),
          |('A', 2, NAMED_STRUCT('row_id', 2, 'json_string', '{"i": 1}')),
          |('A', 2, NAMED_STRUCT('row_id', 2, 'json_string', '{"i": 2}')),
          |('B', 1, NAMED_STRUCT('row_id', 3, 'json_string', '{"i": 1}')),
          |('C', 3, NAMED_STRUCT('row_id', 4, 'json_string', '{"i": 1}'))
        """.stripMargin)

      checkAnswer(
        sql(
          """
            |SELECT a, c.json_string, SUM(b)
            |FROM t
            |GROUP BY a, c.json_string
            |WITH CUBE
            |""".stripMargin),
        Row("A", "{\"i\": 1}", 3) :: Row("A", "{\"i\": 2}", 2) :: Row("A", null, 5) ::
          Row("B", "{\"i\": 1}", 1) :: Row("B", null, 1) ::
          Row("C", "{\"i\": 1}", 3) :: Row("C", null, 3) ::
          Row(null, "{\"i\": 1}", 7) :: Row(null, "{\"i\": 2}", 2) :: Row(null, null, 9) :: Nil)

  }
}
```
Error 
```
[info] - SPARK-31670 *** FAILED *** (2 seconds, 857 milliseconds)
[info]   Failed to analyze query: org.apache.spark.sql.AnalysisException: expression 't.`c`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
[info]   Aggregate [a#247, json_string#248, spark_grouping_id#246L], [a#247, c#223.json_string AS json_string#241, sum(cast(b#222 as bigint)) AS sum(b)#243L]
[info]   +- Expand [List(a#221, b#222, c#223, a#244, json_string#245, 0), List(a#221, b#222, c#223, a#244, null, 1), List(a#221, b#222, c#223, null, json_string#245, 2), List(a#221, b#222, c#223, null, null, 3)], [a#221, b#222, c#223, a#247, json_string#248, spark_grouping_id#246L]
[info]      +- Project [a#221, b#222, c#223, a#221 AS a#244, c#223.json_string AS json_string#245]
[info]         +- SubqueryAlias t
[info]            +- Project [col1#218 AS a#221, col2#219 AS b#222, col3#220 AS c#223]
[info]               +- Project [col1#218, col2#219, col3#220]
[info]                  +- LocalRelation [col1#218, col2#219, col3#220]
[info]
```
For Struct type Field, when we resolve it, it will construct with Alias. When struct field in GROUP BY with CUBE/ROLLUP etc,  struct field in groupByExpression and aggregateExpression will be resolved with different exprId as below
```
'Aggregate [cube(a#221, c#223.json_string AS json_string#240)], [a#221, c#223.json_string AS json_string#241, sum(cast(b#222 as bigint)) AS sum(b)#243L]
+- SubqueryAlias t
   +- Project [col1#218 AS a#221, col2#219 AS b#222, col3#220 AS c#223]
      +- Project [col1#218, col2#219, col3#220]
         +- LocalRelation [col1#218, col2#219, col3#220]
```
This makes `ResolveGroupingAnalytics.constructAggregateExprs()` failed to replace aggreagteExpression use expand groupByExpression attribute since there exprId is not same. then error happened.

### Why are the changes needed?
Fix analyze bug

### Does this PR introduce _any_ user-facing change?
NO

### How was this patch tested?
Added UT

Closes #28490 from AngersZhuuuu/SPARK-31670.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-02 13:49:09 +00:00
Zhenhua Wang 03afbc8820 [SPARK-32739][SQL] Support prune right for left semi join in DPP
### What changes were proposed in this pull request?

Currently in DPP, left semi can only prune left, this pr makes it also support prune right.

### Why are the changes needed?

A minor improvement for DPP.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Add a test case.

Closes #29582 from wzhfy/dpp_support_leftsemi_pruneRight.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2020-09-02 21:34:49 +08:00
Karol Chmist 7511e43c50 [SPARK-32756][SQL] Fix CaseInsensitiveMap usage for Scala 2.13
### What changes were proposed in this pull request?

This is a follow-up of #29160. This allows Spark SQL project to compile for Scala 2.13.

### Why are the changes needed?

It's needed for #28545

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

I compiled with Scala 2.13. It fails in `Spark REPL` project, which will be fixed by #28545

Closes #29584 from karolchmist/SPARK-32364-scala-2.13.

Authored-by: Karol Chmist <info+github@chmist.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-02 08:27:00 -05:00
angerszhu 55ce49ed28 [SPARK-32400][SQL][TEST][FOLLOWUP][TEST-MAVEN] Fix resource loading error in HiveScripTransformationSuite
### What changes were proposed in this pull request?
#29401 move `test_script.py` from sql/hive module to sql/core module, cause HiveScripTransformationSuite load resource issue.

### Why are the changes needed?
This issue cause jenkins test failed in mvn

spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11: https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/
spark-master-test-maven-hadoop-3.2-hive-2.3-jdk-11:
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-hive-2.3-jdk-11/
spark-master-test-maven-hadoop-3.2-hive-2.3:
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-hive-2.3/
![image](https://user-images.githubusercontent.com/46485123/91681585-71285a80-eb81-11ea-8519-99fc9783d6b9.png)

![image](https://user-images.githubusercontent.com/46485123/91681010-aaf86180-eb7f-11ea-8dbb-61365a3b0ab4.png)

Error as below:
```
 Exception thrown while executing Spark plan:
 HiveScriptTransformation [a#349299, b#349300, c#349301, d#349302, e#349303], python /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/hive/file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar!/test_script.py, [a#349309, b#349310, c#349311, d#349312, e#349313], ScriptTransformationIOSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),List((field.delim, )),List((field.delim, )),Some(org.apache.hadoop.hive.ql.exec.TextRecordReader),Some(org.apache.hadoop.hive.ql.exec.TextRecordWriter),false)
+- Project [_1#349288 AS a#349299, _2#349289 AS b#349300, _3#349290 AS c#349301, _4#349291 AS d#349302, _5#349292 AS e#349303]
   +- LocalTableScan [_1#349288, _2#349289, _3#349290, _4#349291, _5#349292]

 == Exception ==
 org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 18021.0 failed 1 times, most recent failure: Lost task 0.0 in stage 18021.0 (TID 37324) (192.168.10.31 executor driver): org.apache.spark.SparkException: Subprocess exited with status 2. Error: python: can't open file '/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/hive/file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar!/test_script.py': [Errno 2] No such file or directory

 at org.apache.spark.sql.execution.BaseScriptTransformationExec.checkFailureAndPropagate(BaseScriptTransformationExec.scala:180)
 at org.apache.spark.sql.execution.BaseScriptTransformationExec.checkFailureAndPropagate$(BaseScriptTransformationExec.scala:157)
 at org.apache.spark.sql.hive.execution.HiveScriptTransformationExec.checkFailureAndPropagate(HiveScriptTransformationExec.scala:49)
 at org.apache.spark.sql.hive.execution.HiveScriptTransformationExec$$anon$1.hasNext(HiveScriptTransformationExec.scala:110)
 at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
 at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
 at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
 at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
 at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
 at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
 at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
 at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
 at org.apache.spark.scheduler.Task.run(Task.scala:127)
 at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:480)
 at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1426)
 at o
```
### Does this PR introduce _any_ user-facing change?
NO

### How was this patch tested?
Existed UT

Closes #29588 from AngersZhuuuu/SPARK-32400-FOLLOWUP.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-02 18:27:29 +09:00
liwensun f0851e95c6 [SPARK-32776][SS] Limit in streaming should not be optimized away by PropagateEmptyRelation
### What changes were proposed in this pull request?

PropagateEmptyRelation will not be applied to LIMIT operators in streaming queries.

### Why are the changes needed?

Right now, the limit operator in a streaming query may get optimized away when the relation is empty. This can be problematic for stateful streaming, as this empty batch will not write any state store files, and the next batch will fail when trying to read these state store files and throw a file not found error.

We should not let PropagateEmptyRelation optimize away the Limit operator for streaming queries.

This PR is intended as a small and safe fix for PropagateEmptyRelation. A fundamental fix that can prevent this from happening again in the future and in other optimizer rules is more desirable, but that's a much larger task.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
unit tests.

Closes #29623 from liwensun/spark-32776.

Authored-by: liwensun <liwen.sun@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-02 18:05:06 +09:00
Yuming Wang 54348dbd21 [SPARK-32767][SQL] Bucket join should work if spark.sql.shuffle.partitions larger than bucket number
### What changes were proposed in this pull request?

Bucket join should work if `spark.sql.shuffle.partitions` larger than bucket number, such as:
```scala
spark.range(1000).write.bucketBy(432, "id").saveAsTable("t1")
spark.range(1000).write.bucketBy(34, "id").saveAsTable("t2")
sql("set spark.sql.shuffle.partitions=600")
sql("set spark.sql.autoBroadcastJoinThreshold=-1")
sql("select * from t1 join t2 on t1.id = t2.id").explain()
```

Before this pr:
```
== Physical Plan ==
*(5) SortMergeJoin [id#26L], [id#27L], Inner
:- *(2) Sort [id#26L ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(id#26L, 600), true
:     +- *(1) Filter isnotnull(id#26L)
:        +- *(1) ColumnarToRow
:           +- FileScan parquet default.t1[id#26L] Batched: true, DataFilters: [isnotnull(id#26L)], Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 432 out of 432
+- *(4) Sort [id#27L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#27L, 600), true
      +- *(3) Filter isnotnull(id#27L)
         +- *(3) ColumnarToRow
            +- FileScan parquet default.t2[id#27L] Batched: true, DataFilters: [isnotnull(id#27L)], Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 34 out of 34
```

After this pr:
```
== Physical Plan ==
*(4) SortMergeJoin [id#26L], [id#27L], Inner
:- *(1) Sort [id#26L ASC NULLS FIRST], false, 0
:  +- *(1) Filter isnotnull(id#26L)
:     +- *(1) ColumnarToRow
:        +- FileScan parquet default.t1[id#26L] Batched: true, DataFilters: [isnotnull(id#26L)], Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 432 out of 432
+- *(3) Sort [id#27L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#27L, 432), true
      +- *(2) Filter isnotnull(id#27L)
         +- *(2) ColumnarToRow
            +- FileScan parquet default.t2[id#27L] Batched: true, DataFilters: [isnotnull(id#27L)], Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 34 out of 34
```

### Why are the changes needed?

Spark 2.4 support this.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Unit test.

Closes #29612 from wangyum/SPARK-32767.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-02 04:16:20 +00:00
Kousuke Saruta 812d0918a8 [SPARK-32771][DOCS] The example of expressions.Aggregator in Javadoc / Scaladoc is wrong
### What changes were proposed in this pull request?

This PR modifies an example for `expressions.Aggregator` in Javadoc and Scaladoc.
The definition of `bufferEncoder` and `outputEncoder` are added.

### Why are the changes needed?

To correct the example.
The current example is wrong and doesn't work because `bufferEncoder` and `outputEncoder` are not defined.

### Does this PR introduce _any_ user-facing change?

Yes.
Before this change, the scaladoc and javadoc are like as follows.
![wrong-example-java](https://user-images.githubusercontent.com/4736016/91897528-5ebf3580-ecd5-11ea-8d7b-e846b776ebbb.png)
![wrong-example](https://user-images.githubusercontent.com/4736016/91897509-58c95480-ecd5-11ea-81a3-98774083b689.png)

After this change, the docs are like as follows.
![fixed-example-java](https://user-images.githubusercontent.com/4736016/91897592-78607d00-ecd5-11ea-9e55-03fd9c9c6b54.png)
![fixed-example](https://user-images.githubusercontent.com/4736016/91897609-7c8c9a80-ecd5-11ea-837e-9dbcada6cd53.png)

### How was this patch tested?

Build with `build/sbt unidoc` and confirmed the generated javadoc/scaladoc and got the screenshots above.

Closes #29617 from sarutak/fix-aggregator-doc.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-02 10:03:07 +09:00
Linhong Liu a410658c9b [SPARK-32761][SQL] Allow aggregating multiple foldable distinct expressions
### What changes were proposed in this pull request?
For queries with multiple foldable distinct columns, since they will be eliminated during
execution, it's not mandatory to let `RewriteDistinctAggregates` handle this case. And
in the current code, `RewriteDistinctAggregates` *dose* miss some "aggregating with
multiple foldable distinct expressions" cases.
For example: `select count(distinct 2), count(distinct 2, 3)` will be missed.

But in the planner, this will trigger an error that "multiple distinct expressions" are not allowed.
As the foldable distinct columns can be eliminated finally, we can allow this in the aggregation
planner check.

### Why are the changes needed?
bug fix

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
added test case

Closes #29607 from linhongliu-db/SPARK-32761.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-01 13:04:24 +00:00
Wenchen Fan fea9360ae7 [SPARK-32757][SQL][FOLLOW-UP] Use child's output for canonicalization in SubqueryBroadcastExec
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/29601 , to fix a small mistake in `SubqueryBroadcastExec`. `SubqueryBroadcastExec.doCanonicalize` should canonicalize the build keys with the query output, not the `SubqueryBroadcastExec.output`.

### Why are the changes needed?

fix mistake

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

existing test

Closes #29610 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-01 12:54:40 +00:00
Huaxin Gao e1dbc85c72 [SPARK-32579][SQL] Implement JDBCScan/ScanBuilder/WriteBuilder
### What changes were proposed in this pull request?
Add JDBCScan, JDBCScanBuilder, JDBCWriteBuilder in Datasource V2 JDBC

### Why are the changes needed?
Complete Datasource V2 JDBC implementation

### Does this PR introduce _any_ user-facing change?
Yes

### How was this patch tested?
new tests

Closes #29396 from huaxingao/v2jdbc.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-01 07:23:20 +00:00
Wenchen Fan d2a5dad97c [SPARK-32757][SQL] Physical InSubqueryExec should be consistent with logical InSubquery
### What changes were proposed in this pull request?

`InSubquery` can be either single-column mode, or multi-column mode, depending on the output length of the subquery. For multi-column mode, the length of input `values` must match the subquery output length.

However, `InSubqueryExec` doesn't follow it and always be executed under single column mode. It's OK as it's only used by DPP, which looks up one key in one `InSubqueryExec`, so the multi-column mode is not needed. But it's better to make the physical and logical node consistent.

This PR updates `InSubqueryExec` to support multi-column mode, and also fix `SubqueryBroadcastExec` to report output correctly.

### Why are the changes needed?

Fix a potential bug.

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

existing tests

Closes #29601 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-01 07:19:43 +00:00
Yuming Wang a701bc79e3 [SPARK-32659][SQL][FOLLOWUP] Improve test for pruning DPP on non-atomic type
### What changes were proposed in this pull request?

Improve test for pruning DPP on non-atomic type:
- Avoid creating new partition tables. This may take 30 seconds..
- Add test `array` type.

### Why are the changes needed?

Improve test.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

N/A

Closes #29595 from wangyum/SPARK-32659-test.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-01 05:51:04 +00:00
Huaxin Gao 806140de40 [SPARK-32592][SQL] Make DataFrameReader.table take the specified options
### What changes were proposed in this pull request?
pass specified options in DataFrameReader.table to JDBCTableCatalog.loadTable

### Why are the changes needed?
Currently, `DataFrameReader.table` ignores the specified options. The options specified like the following are lost.
```
    val df = spark.read
      .option("partitionColumn", "id")
      .option("lowerBound", "0")
      .option("upperBound", "3")
      .option("numPartitions", "2")
      .table("h2.test.people")
```
We need to make `DataFrameReader.table` take the specified options.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manually test for now. Will add a test after V2 JDBC read is implemented.

Closes #29535 from huaxingao/table_options.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-31 13:21:15 +00:00
Cheng Su ce473b223a [SPARK-32740][SQL] Refactor common partitioning/distribution logic to BaseAggregateExec
### What changes were proposed in this pull request?

For all three different aggregate physical operator: `HashAggregateExec`, `ObjectHashAggregateExec` and `SortAggregateExec`, they have same `outputPartitioning` and `requiredChildDistribution` logic. Refactor these same logic into their super class `BaseAggregateExec` to avoid code duplication and future bugs (similar to `HashJoin` and `ShuffledJoin`).

### Why are the changes needed?

Reduce duplicated code across classes and prevent future bugs if we only update one class but forget another. We already did similar refactoring for join (`HashJoin` and `ShuffledJoin`).

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing unit tests as this is pure refactoring and no new logic added.

Closes #29583 from c21/aggregate-refactor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-31 15:43:13 +09:00
Udbhav30 065f17386d [SPARK-32481][CORE][SQL] Support truncate table to move data to trash
### What changes were proposed in this pull request?
Instead of deleting the data, we can move the data to trash.
Based on the configuration provided by the user it will be deleted permanently from the trash.

### Why are the changes needed?
Instead of directly deleting the data, we can provide flexibility to move data to the trash and then delete it permanently.

### Does this PR introduce _any_ user-facing change?
Yes, After truncate table the data is not permanently deleted now.
It is first moved to the trash and then after the given time deleted permanently;

### How was this patch tested?
new UTs added

Closes #29552 from Udbhav30/truncate.

Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-30 10:25:32 -07:00
Cheng Su cfe012a431 [SPARK-32629][SQL] Track metrics of BitSet/OpenHashSet in full outer SHJ
### What changes were proposed in this pull request?

This is followup from https://github.com/apache/spark/pull/29342, where to do two things:
* Per https://github.com/apache/spark/pull/29342#discussion_r470153323, change from java `HashSet` to spark in-house `OpenHashSet` to track matched rows for non-unique join keys. I checked `OpenHashSet` implementation which is built from a key index (`OpenHashSet._bitset` as `BitSet`) and key array (`OpenHashSet._data` as `Array`). Java `HashSet` is built from `HashMap`, which stores value in `Node` linked list and by theory should have taken more memory than `OpenHashSet`. Reran the same benchmark query used in https://github.com/apache/spark/pull/29342, and verified the query has similar performance here between `HashSet` and `OpenHashSet`.
* Track metrics of the extra data structure `BitSet`/`OpenHashSet` for full outer SHJ. This depends on above thing, because there seems no easy way to get java `HashSet` memory size.

### Why are the changes needed?

To better surface the memory usage for full outer SHJ more accurately.
This can help users/developers to debug/improve full outer SHJ.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added unite test in `SQLMetricsSuite.scala` .

Closes #29566 from c21/add-metrics.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-30 07:01:33 +09:00
Wenchen Fan ccc0250a08 [SPARK-32718][SQL] Remove unnecessary keywords for interval units
### What changes were proposed in this pull request?

Remove the YEAR, MONTH, DAY, HOUR, MINUTE, SECOND keywords. They are not useful in the parser, as we need to support plural like YEARS, so the parser has to accept the general identifier as interval unit anyway.

### Why are the changes needed?

These keywords are reserved in ANSI. If Spark has these keywords, then they become reserved under ANSI mode. This makes Spark not able to run TPCDS queries as they use YEAR as alias name.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added `TPCDSQueryANSISuite`, to make sure Spark with ANSI mode can run TPCDS queries.

Closes #29560 from cloud-fan/keyword.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-29 14:06:01 -07:00
Chen Zhang 58f87b3178 [SPARK-32639][SQL] Support GroupType parquet mapkey field
### What changes were proposed in this pull request?
Remove the assertion in ParquetSchemaConverter that the parquet mapKey field must be PrimitiveType.

### Why are the changes needed?
There is a parquet file in the attachment of [SPARK-32639](https://issues.apache.org/jira/browse/SPARK-32639), and the MessageType recorded in the file is:
```
message parquet_schema {
  optional group value (MAP) {
    repeated group key_value {
      required group key {
        optional binary first (UTF8);
        optional binary middle (UTF8);
        optional binary last (UTF8);
      }
      optional binary value (UTF8);
    }
  }
}
```

Use `spark.read.parquet("000.snappy.parquet")` to read the file. Spark will throw an exception when converting Parquet MessageType to Spark SQL StructType:

> AssertionError(Map key type is expected to be a primitive type, but found...)

Use `spark.read.schema("value MAP<STRUCT<first:STRING, middle:STRING, last:STRING>, STRING>").parquet("000.snappy.parquet")` to read the file, spark returns the correct result .

According to the parquet project document (https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#maps), the mapKey in the parquet format does not need to be a primitive type.

Note: This parquet file is not written by spark, because spark will write additional sparkSchema string information in the parquet file. When Spark reads, it will directly use the additional sparkSchema information in the file instead of converting Parquet MessageType to Spark SQL StructType.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added a unit test case

Closes #29451 from izchen/SPARK-32639.

Authored-by: Chen Zhang <izchen@126.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-28 16:51:00 +00:00
Takeshi Yamamuro 0cb91b8c18 [SPARK-32704][SQL] Logging plan changes for execution
### What changes were proposed in this pull request?

Since we only log plan changes for analyzer/optimizer now, this PR intends to add code to log plan changes in the preparation phase in `QueryExecution` for execution.
```
scala> spark.sql("SET spark.sql.optimizer.planChangeLog.level=WARN")
scala> spark.range(10).groupBy("id").count().queryExecution.executedPlan
...
20/08/26 09:32:36 WARN PlanChangeLogger:
=== Applying Rule org.apache.spark.sql.execution.CollapseCodegenStages ===
!HashAggregate(keys=[id#19L], functions=[count(1)], output=[id#19L, count#23L])              *(1) HashAggregate(keys=[id#19L], functions=[count(1)], output=[id#19L, count#23L])
!+- HashAggregate(keys=[id#19L], functions=[partial_count(1)], output=[id#19L, count#27L])   +- *(1) HashAggregate(keys=[id#19L], functions=[partial_count(1)], output=[id#19L, count#27L])
!   +- Range (0, 10, step=1, splits=4)                                                          +- *(1) Range (0, 10, step=1, splits=4)

20/08/26 09:32:36 WARN PlanChangeLogger:
=== Result of Batch Preparations ===
!HashAggregate(keys=[id#19L], functions=[count(1)], output=[id#19L, count#23L])              *(1) HashAggregate(keys=[id#19L], functions=[count(1)], output=[id#19L, count#23L])
!+- HashAggregate(keys=[id#19L], functions=[partial_count(1)], output=[id#19L, count#27L])   +- *(1) HashAggregate(keys=[id#19L], functions=[partial_count(1)], output=[id#19L, count#27L])
!   +- Range (0, 10, step=1, splits=4)                                                          +- *(1) Range (0, 10, step=1, splits=4)
```

### Why are the changes needed?

Easy debugging for executed plans

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added unit tests.

Closes #29544 from maropu/PlanLoggingInPreparations.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-28 16:35:47 +00:00
yi.wu c3b9404253 [SPARK-32717][SQL] Add a AQEOptimizer for AdaptiveSparkPlanExec
### What changes were proposed in this pull request?

This PR proposes to add a specific `AQEOptimizer` for the `AdaptiveSparkPlanExec` instead of implementing an anonymous `RuleExecutor`. At the same time, this PR also adds the configuration `spark.sql.adaptive.optimizer.excludedRules`, which follows the same pattern of `Optimizer`, to make the `AQEOptimizer` more flexible for users and developers.

### Why are the changes needed?

Currently, `AdaptiveSparkPlanExec` has implemented an anonymous `RuleExecutor` to apply the AQE optimize rules on the plan. However, the anonymous class usually could be inconvenient to maintain and extend for the long term.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

It's a pure refactor so pass existing tests should be ok.

Closes #29559 from Ngone51/impro-aqe-optimizer.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-28 21:23:53 +09:00
Jungtaek Lim (HeartSaVioR) 73bfed3633 [SPARK-28612][SQL][FOLLOWUP] Correct method doc of DataFrameWriterV2.replace()
### What changes were proposed in this pull request?

This patch corrects the method doc of DataFrameWriterV2.replace() which explanation of exception is described oppositely.

### Why are the changes needed?

The method doc is incorrect.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Only doc change.

Closes #29568 from HeartSaVioR/SPARK-28612-FOLLOWUP-fix-doc-nit.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-28 15:14:57 +09:00
Liang-Chi Hsieh d6c095c92c [SPARK-32693][SQL] Compare two dataframes with same schema except nullable property
### What changes were proposed in this pull request?

This PR changes key data types check in `HashJoin` to use `sameType`.

### Why are the changes needed?

Looks at the resolving condition of `SetOperation`, it requires only each left data types should be `sameType` as the right ones. Logically the `EqualTo` expression in equi-join, also requires only left data type `sameType` as right data type. Then `HashJoin` requires left keys data type exactly the same as right keys data type, looks not reasonable.

It makes inconsistent results when doing `except` between two dataframes.

If two dataframes don't have nested fields, even their field nullable property different, `HashJoin` passes the key type check because it checks field individually so field nullable property is ignored.

If two dataframes have nested fields like struct, `HashJoin` fails the key type check because now it compare two struct types and nullable property now affects.

### Does this PR introduce _any_ user-facing change?

Yes. Making consistent `except` operation between dataframes.

### How was this patch tested?

Unit test.

Closes #29555 from viirya/SPARK-32693.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-28 10:32:23 +09:00
xuewei.linxuewei eb379766f4 [SPARK-32705][SQL] Fix serialization issue for EmptyHashedRelation
### What changes were proposed in this pull request?
Currently, EmptyHashedRelation and HashedRelationWithAllNullKeys is an object, and it will cause JavaDeserialization Exception as following

```
20/08/26 11:13:30 WARN [task-result-getter-2] TaskSetManager: Lost task 34.0 in stage 57.0 (TID 18076, emr-worker-5.cluster-183257, executor 18): java.io.InvalidClassException: org.apache.spark.sql.execution.joins.EmptyHashedRelation$; no valid constructor
        at java.io.ObjectStreamClass$ExceptionInfo.newInvalidClassException(ObjectStreamClass.java:169)
        at java.io.ObjectStreamClass.checkDeserialize(ObjectStreamClass.java:874)
        at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2042)
        at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1572)
        at java.io.ObjectInputStream.readObject(ObjectInputStream.java:430)
        at org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76)
        at org.apache.spark.broadcast.TorrentBroadcast$.$anonfun$unBlockifyObject$4(TorrentBroadcast.scala:328)
```

This PR includes

* Using case object instead to fix serialization issue.
* Also change EmptyHashedRelation not to extend NullAwareHashedRelation since it's already being used in other non-NAAJ joins.

### Why are the changes needed?
It will cause BHJ failed when buildSide is Empty and BHJ(NAAJ) failed when buildSide with null partition keys.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
* Existing UT.
* Run entire TPCDS for E2E coverage.

Closes #29547 from leanken/leanken-SPARK-32705.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-27 06:24:42 +00:00
Terry Kim baaa756dee [SPARK-32516][SQL][FOLLOWUP] 'path' option cannot coexist with path parameter for DataFrameWriter.save(), DataStreamReader.load() and DataStreamWriter.start()
### What changes were proposed in this pull request?

This is a follow up PR to #29328 to apply the same constraint where `path` option cannot coexist with path parameter to `DataFrameWriter.save()`, `DataStreamReader.load()` and `DataStreamWriter.start()`.

### Why are the changes needed?

The current behavior silently overwrites the `path` option if path parameter is passed to `DataFrameWriter.save()`, `DataStreamReader.load()` and `DataStreamWriter.start()`.

For example,
```
Seq(1).toDF.write.option("path", "/tmp/path1").parquet("/tmp/path2")
```
will write the result to `/tmp/path2`.

### Does this PR introduce _any_ user-facing change?

Yes, if `path` option coexists with path parameter to any of the above methods, it will throw `AnalysisException`:
```
scala> Seq(1).toDF.write.option("path", "/tmp/path1").parquet("/tmp/path2")
org.apache.spark.sql.AnalysisException: There is a 'path' option set and save() is called with a  path parameter. Either remove the path option, or call save() without the parameter. To ignore this check, set 'spark.sql.legacy.pathOptionBehavior.enabled' to 'true'.;
```

The user can restore the previous behavior by setting `spark.sql.legacy.pathOptionBehavior.enabled` to `true`.

### How was this patch tested?

Added new tests.

Closes #29543 from imback82/path_option.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-27 06:21:04 +00:00
Dongjoon Hyun 2dee4352a0 Revert "[SPARK-32481][CORE][SQL] Support truncate table to move data to trash"
This reverts commit 5c077f0580.
2020-08-26 11:24:35 -07:00
Yuming Wang a8b568800e [SPARK-32659][SQL] Fix the data issue when pruning DPP on non-atomic type
### What changes were proposed in this pull request?

Use `InSet` expression to fix data issue when pruning DPP on non-atomic type. for example:
   ```scala
    spark.range(1000)
    .select(col("id"), col("id").as("k"))
    .write
    .partitionBy("k")
    .format("parquet")
    .mode("overwrite")
    .saveAsTable("df1");

   spark.range(100)
   .select(col("id"), col("id").as("k"))
   .write
   .partitionBy("k")
   .format("parquet")
   .mode("overwrite")
   .saveAsTable("df2")

   spark.sql("set spark.sql.optimizer.dynamicPartitionPruning.fallbackFilterRatio=2")
   spark.sql("set spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly=false")
   spark.sql("SELECT df1.id, df2.k FROM df1 JOIN df2 ON struct(df1.k) = struct(df2.k) AND df2.id < 2").show
   ```
   It should return two records, but it returns empty.

### Why are the changes needed?

Fix data issue

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Add new unit test.

Closes #29475 from wangyum/SPARK-32659.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-26 06:57:43 +00:00
Udbhav30 5c077f0580 [SPARK-32481][CORE][SQL] Support truncate table to move data to trash
### What changes were proposed in this pull request?
Instead of deleting the data, we can move the data to trash.
Based on the configuration provided by the user it will be deleted permanently from the trash.

### Why are the changes needed?
Instead of directly deleting the data, we can provide flexibility to move data to the trash and then delete it permanently.

### Does this PR introduce _any_ user-facing change?
Yes, After truncate table the data is not permanently deleted now.
It is first moved to the trash and then after the given time deleted permanently;

### How was this patch tested?
new UTs added

Closes #29387 from Udbhav30/tuncateTrash.

Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-25 23:38:43 -07:00
yi.wu f510d21e93 [SPARK-32466][FOLLOW-UP][TEST][SQL] Regenerate the golden explain file for PlanStabilitySuite
### What changes were proposed in this pull request?

This PR regenerates the golden explain file based on the fix: https://github.com/apache/spark/pull/29537

### Why are the changes needed?

Eliminates the personal related information (e.g., local directories) in the explain plan.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Checked manually.

Closes #29546 from Ngone51/follow-up-gen-golden-file.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-26 14:46:14 +09:00
Yuming Wang 1354cf0842 [SPARK-32620][SQL] Reset the numPartitions metric when DPP is enabled
### What changes were proposed in this pull request?

This pr reset the `numPartitions` metric when DPP is enabled. Otherwise, it is always a [static value](18cac6a9f0/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala (L215)).

### Why are the changes needed?

Fix metric issue.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Unit test and manual test

For [this test case](18cac6a9f0/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala (L252-L280)).

Before this pr:
![image](https://user-images.githubusercontent.com/5399861/90301798-9310b480-ded4-11ea-9294-49bcaba46f83.png)

After this pr:
![image](https://user-images.githubusercontent.com/5399861/90301709-0fef5e80-ded4-11ea-942d-4d45d1dd15bc.png)

Closes #29436 from wangyum/SPARK-32620.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2020-08-25 18:46:10 -07:00
Sean Owen a9d4e60a90 [SPARK-32614][SQL] Don't apply comment processing if 'comment' unset for CSV
### What changes were proposed in this pull request?

Spark's CSV source can optionally ignore lines starting with a comment char. Some code paths check to see if it's set before applying comment logic (i.e. not set to default of `\0`), but many do not, including the one that passes the option to Univocity. This means that rows beginning with a null char were being treated as comments even when 'disabled'.

### Why are the changes needed?

To avoid dropping rows that start with a null char when this is not requested or intended. See JIRA for an example.

### Does this PR introduce _any_ user-facing change?

Nothing beyond the effect of the bug fix.

### How was this patch tested?

Existing tests plus new test case.

Closes #29516 from srowen/SPARK-32614.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-26 00:25:58 +09:00
yi.wu b78b776c9e [SPARK-32466][SQL][FOLLOW-UP] Normalize Location info in explain plan
### What changes were proposed in this pull request?

1. Extract `SQLQueryTestSuite.replaceNotIncludedMsg` to `PlanTest`.

2. Reuse `replaceNotIncludedMsg` to normalize the explain plan that generated in `PlanStabilitySuite`.

### Why are the changes needed?

This's a follow-up of https://github.com/apache/spark/pull/29270.
Eliminates the personal related information (e.g., local directories) in the explain plan.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Updated test.

Closes #29537 from Ngone51/follow-up-plan-stablity.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-25 21:03:44 +09:00
fqaiser94@gmail.com 3f1e56d4ca [SPARK-32641][SQL] withField + getField should return null if original struct was null
### What changes were proposed in this pull request?

There is a bug in the way the optimizer rule in `SimplifyExtractValueOps` is currently written in master branch which yields incorrect results in scenarios like the following:
```
sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
.select($"struct_col".withField("d", lit(4)).getField("d").as("d"))

// currently returns this:
+---+
|d  |
+---+
|4  |
+---+

// when in fact it should return this:
+----+
|d   |
+----+
|null|
+----+
```
The changes in this PR will fix this bug.

### Why are the changes needed?

To fix the aforementioned bug. Optimizer rules should improve the performance of the  query but yield exactly the same results.

### Does this PR introduce _any_ user-facing change?

Yes, this bug will no longer occur.
That said, this isn't something to be concerned about as this bug was introduced in Spark 3.1 and Spark 3.1 has yet to be released.

### How was this patch tested?

Unit tests were added. Jenkins must pass them.

Closes #29522 from fqaiser94/SPARK-32641.

Authored-by: fqaiser94@gmail.com <fqaiser94@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-25 04:59:37 +00:00
Liang-Chi Hsieh cee48a9661 [SPARK-32646][SQL][TEST-HADOOP2.7][TEST-HIVE1.2] ORC predicate pushdown should work with case-insensitive analysis
### What changes were proposed in this pull request?

This PR proposes to fix ORC predicate pushdown under case-insensitive analysis case. The field names in pushed down predicates don't need to match in exact letter case with physical field names in ORC files, if we enable case-insensitive analysis.

This is re-submitted for #29457.  Because #29457 has a hive-1.2 error and there were some tests failed with hive-1.2 profile at the same time, #29457 was reverted to unblock others.

### Why are the changes needed?

Currently ORC predicate pushdown doesn't work with case-insensitive analysis. A predicate "a < 0" cannot pushdown to ORC file with field name "A" under case-insensitive analysis.

But Parquet predicate pushdown works with this case. We should make ORC predicate pushdown work with case-insensitive analysis too.

### Does this PR introduce _any_ user-facing change?

Yes, after this PR, under case-insensitive analysis, ORC predicate pushdown will work.

### How was this patch tested?

Unit tests.

Closes #29530 from viirya/fix-orc-pushdown.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-25 13:47:52 +09:00
Nicholas Chammas f540031419 [SPARK-31000][PYTHON][SQL] Add ability to set table description via Catalog.createTable()
### What changes were proposed in this pull request?

This PR enhances `Catalog.createTable()` to allow users to set the table's description. This corresponds to the following SQL syntax:

```sql
CREATE TABLE ...
COMMENT 'this is a fancy table';
```

### Why are the changes needed?

This brings the Scala/Python catalog APIs a bit closer to what's already possible via SQL.

### Does this PR introduce any user-facing change?

Yes, it adds a new parameter to `Catalog.createTable()`.

### How was this patch tested?

Existing unit tests:

```sh
./python/run-tests \
  --python-executables python3.7 \
  --testnames 'pyspark.sql.tests.test_catalog,pyspark.sql.tests.test_context'
```

```
$ ./build/sbt
testOnly org.apache.spark.sql.internal.CatalogSuite org.apache.spark.sql.CachedTableSuite org.apache.spark.sql.hive.MetastoreDataSourcesSuite org.apache.spark.sql.hive.execution.HiveDDLSuite
```

Closes #27908 from nchammas/SPARK-31000-table-description.

Authored-by: Nicholas Chammas <nicholas.chammas@liveramp.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-25 13:42:31 +09:00
Terry Kim e3a88a9767 [SPARK-32516][SQL] 'path' option cannot coexist with load()'s path parameters
### What changes were proposed in this pull request?

This PR proposes to make the behavior consistent for the `path` option when loading dataframes with a single path (e.g, `option("path", path).format("parquet").load(path)` vs. `option("path", path).parquet(path)`) by disallowing `path` option to coexist with `load`'s path parameters.

### Why are the changes needed?

The current behavior is inconsistent:
```scala
scala> Seq(1).toDF.write.mode("overwrite").parquet("/tmp/test")

scala> spark.read.option("path", "/tmp/test").format("parquet").load("/tmp/test").show
+-----+
|value|
+-----+
|    1|
+-----+

scala> spark.read.option("path", "/tmp/test").parquet("/tmp/test").show
+-----+
|value|
+-----+
|    1|
|    1|
+-----+
```

### Does this PR introduce _any_ user-facing change?

Yes, now if the `path` option is specified along with `load`'s path parameters, it would fail:
```scala
scala> Seq(1).toDF.write.mode("overwrite").parquet("/tmp/test")

scala> spark.read.option("path", "/tmp/test").format("parquet").load("/tmp/test").show
org.apache.spark.sql.AnalysisException: There is a path option set and load() is called with path parameters. Either remove the path option or move it into the load() parameters.;
  at org.apache.spark.sql.DataFrameReader.verifyPathOptionDoesNotExist(DataFrameReader.scala:310)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:232)
  ... 47 elided

scala> spark.read.option("path", "/tmp/test").parquet("/tmp/test").show
org.apache.spark.sql.AnalysisException: There is a path option set and load() is called with path parameters. Either remove the path option or move it into the load() parameters.;
  at org.apache.spark.sql.DataFrameReader.verifyPathOptionDoesNotExist(DataFrameReader.scala:310)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:250)
  at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:778)
  at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:756)
  ... 47 elided
```

The user can restore the previous behavior by setting `spark.sql.legacy.pathOptionBehavior.enabled` to `true`.

### How was this patch tested?

Added a test

Closes #29328 from imback82/dfw_option.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-24 16:30:30 +00:00
Cheng Su 08b951b1cb [SPARK-32649][SQL] Optimize BHJ/SHJ inner/semi join with empty hashed relation
### What changes were proposed in this pull request?

For broadcast hash join and shuffled hash join, whenever the build side hashed relation turns out to be empty. We don't need to execute stream side plan at all, and can return an empty iterator (for inner join and left semi join), because we know for sure that none of stream side rows can be outputted as there's no match.

### Why are the changes needed?

A very minor optimization for rare use case, but in case build side turns out to be empty, we can leverage it to short-cut stream side to save CPU and IO.

Example broadcast hash join query similar to `JoinBenchmark` with empty hashed relation:

```
  def broadcastHashJoinLongKey(): Unit = {
    val N = 20 << 20
    val M = 1 << 16

    val dim = broadcast(spark.range(0).selectExpr("id as k", "cast(id as string) as v"))
    codegenBenchmark("Join w long", N) {
      val df = spark.range(N).join(dim, (col("id") % M) === col("k"))
      assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined)
      df.noop()
    }
  }
```

Comparing wall clock time for enabling and disabling this PR (for non-codegen code path). Seeing like 8x improvement.

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Join w long:                              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
Join PR disabled                                    637            646          12         32.9          30.4       1.0X
Join PR enabled                                      77             78           2        271.8           3.7       8.3X
```

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added unit test in `JoinSuite`.

Closes #29484 from c21/empty-relation.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-24 08:53:26 +00:00
Liang-Chi Hsieh 11c6a23c13 [SPARK-32352][SQL][FOLLOW-UP][TEST-HADOOP2.7][TEST-HIVE1.2] Exclude partition columns from data columns
### What changes were proposed in this pull request?

This PR fixes a bug of #29406. #29406 partially pushes down data filter even if it mixed in partition filters. But in some cases partition columns might be in data columns too. It will possibly push down a predicate with partition column to datasource.

### Why are the changes needed?

The test "org.apache.spark.sql.hive.orc.HiveOrcHadoopFsRelationSuite.save()/load() - partitioned table - simple queries - partition columns in data" is currently failed with hive-1.2 profile in master branch.

```
[info] - save()/load() - partitioned table - simple queries - partition columns in data *** FAILED *** (1 second, 457 milliseconds)
[info]   java.util.NoSuchElementException: key not found: p1
[info]   at scala.collection.immutable.Map$Map2.apply(Map.scala:138)
[info]   at org.apache.spark.sql.hive.orc.OrcFilters$.buildLeafSearchArgument(OrcFilters.scala:250)
[info]   at org.apache.spark.sql.hive.orc.OrcFilters$.convertibleFiltersHelper$1(OrcFilters.scala:143)
[info]   at org.apache.spark.sql.hive.orc.OrcFilters$.$anonfun$convertibleFilters$4(OrcFilters.scala:146)
[info]   at scala.collection.TraversableLike.$anonfun$flatMap$1(TraversableLike.scala:245)
[info]   at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
[info]   at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
[info]   at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:38)
[info]   at scala.collection.TraversableLike.flatMap(TraversableLike.scala:245)
[info]   at scala.collection.TraversableLike.flatMap$(TraversableLike.scala:242)
[info]   at scala.collection.AbstractTraversable.flatMap(Traversable.scala:108)
[info]   at org.apache.spark.sql.hive.orc.OrcFilters$.convertibleFilters(OrcFilters.scala:145)
[info]   at org.apache.spark.sql.hive.orc.OrcFilters$.createFilter(OrcFilters.scala:83)
[info]   at org.apache.spark.sql.hive.orc.OrcFileFormat.buildReader(OrcFileFormat.scala:142)
```

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Unit test.

Closes #29526 from viirya/SPARK-32352-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-24 14:06:08 +09:00
mingjial b9585cde31 [SPARK-32609][TEST] Add Tests for Incorrect exchange reuse with DataSourceV2
### What changes were proposed in this pull request?
Copy  to master branch the unit test added for branch-2.4(https://github.com/apache/spark/pull/29430).

### Why are the changes needed?
The unit test will pass at master branch, indicating that issue reported in https://issues.apache.org/jira/browse/SPARK-32609 is already fixed at master branch. But adding this unit test for future possible failure catch.

### Does this PR introduce _any_ user-facing change?
no.

### How was this patch tested?
sbt test run

Closes #29435 from mingjialiu/master.

Authored-by: mingjial <mingjial@google.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-23 17:40:59 -07:00
majsiel 772c70615a [SPARK-32648][SS] Remove unused DELETE_ACTION in FileStreamSinkLog
### What changes were proposed in this pull request?
Removing unused DELETE_ACTION in FileStreamSinkLog.

### Why are the changes needed?
DELETE_ACTION is not used nowhere in the code.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Tests where not added, because code was removed.

Closes #29505 from michal-wieleba/SPARK-32648.

Authored-by: majsiel <majsiel@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-23 17:17:39 -07:00
Max Gekk 1c798f973f [SPARK-32594][SQL][FOLLOWUP][TEST-HADOOP2.7][TEST-HIVE1.2] Override get() and use Julian days in DaysWritable
### What changes were proposed in this pull request?
Override `def get: Date` in `DaysWritable` use the `daysToMillis(int d)` from the parent class `DateWritable` instead of `long daysToMillis(int d, boolean doesTimeMatter)`.

### Why are the changes needed?
It fixes failures of `HiveSerDeReadWriteSuite` with the profile `hive-1.2`. In that case, the parent class `DateWritable` has different implementation before the commit to Hive da3ed68eda. In particular, `get()` calls `new Date(daysToMillis(daysSinceEpoch))` instead of overrided `def get(doesTimeMatter: Boolean): Date` in the child class. The `get()` method returns wrong result `1970-01-01` because it uses not updated `daysSinceEpoch`.

### Does this PR introduce _any_ user-facing change?
Yes.

### How was this patch tested?
By running the test suite `HiveSerDeReadWriteSuite`:
```
$ build/sbt -Phive-1.2 -Phadoop-2.7 "test:testOnly org.apache.spark.sql.hive.execution.HiveSerDeReadWriteSuite"
```
and
```
$ build/sbt -Phive-2.3 -Phadoop-2.7 "test:testOnly org.apache.spark.sql.hive.execution.HiveSerDeReadWriteSuite"
```

Closes #29523 from MaxGekk/insert-date-into-hive-table-1.2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-08-23 12:43:30 -07:00
angerszhu 9808c15eec [SPARK-32608][SQL][FOLLOW-UP][TEST-HADOOP2.7][TEST-HIVE1.2] Script Transform ROW FORMAT DELIMIT value should format value
### What changes were proposed in this pull request?
As mentioned in https://github.com/apache/spark/pull/29428#issuecomment-678735163 by viirya ,
fix bug in UT, since in script transformation no-serde mode, output of decimal is same in both hive-1.2/hive-2.3

### Why are the changes needed?
FIX UT

### Does this PR introduce _any_ user-facing change?
NO

### How was this patch tested?
EXISTED UT

Closes #29520 from AngersZhuuuu/SPARK-32608-FOLLOW.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-08-23 08:08:55 -07:00
Liang-Chi Hsieh aa0b0b87e2 Revert "[SPARK-32646][SQL] ORC predicate pushdown should work with case-insensitive analysis"
### What changes were proposed in this pull request?

This reverts commit e277ef1a83.

### Why are the changes needed?

Because master and branch-3.0 both have few tests failed under hive-1.2 profile. And the PR #29457 missed a change in hive-1.2 code that causes compilation error. So it will make debugging the failed tests harder. I'd like revert #29457 first to unblock it.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Unit test

Closes #29519 from viirya/revert-SPARK-32646.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-08-23 00:40:53 -07:00
xuewei.linxuewei f258718535 [SPARK-32678][SQL] Rename EmptyHashedRelationWithAllNullKeys and simplify NAAJ generated code
### What changes were proposed in this pull request?
Some Code refine.

1. rename EmptyHashedRelationWithAllNullKeys to HashedRelationWithAllNullKeys.
2. simplify generated code for BHJ NAAJ.

### Why are the changes needed?
Refine code and naming to avoid confusing understanding.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Existing test.

Closes #29503 from leanken/leanken-SPARK-32678.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-08-22 22:32:39 -07:00
Yuanjian Li 8b26c69ce7 [SPARK-31792][SS][DOC][FOLLOW-UP] Rephrase the description for some operations
### What changes were proposed in this pull request?
Rephrase the description for some operations to make it clearer.

### Why are the changes needed?
Add more detail in the document.

### Does this PR introduce _any_ user-facing change?
No, document only.

### How was this patch tested?
Document only.

Closes #29269 from xuanyuanking/SPARK-31792-follow.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-08-22 21:32:23 +09:00
Robert (Bobby) Evans 12f4331b9e [SPARK-32672][SQL] Fix data corruption in boolean bit set compression
## What changes were proposed in this pull request?

This fixed SPARK-32672 a data corruption.  Essentially the BooleanBitSet CompressionScheme would miss nulls at the end of a CompressedBatch.  The values would then default to false.

### Why are the changes needed?
It fixes data corruption

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
I manually tested it against the original issue that was producing errors for me.  I also added in a unit test.

Closes #29506 from revans2/SPARK-32672.

Authored-by: Robert (Bobby) Evans <bobby@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-22 11:07:14 +09:00
Brandon Jiang 1450b5e095 [MINOR][DOCS] fix typo for docs,log message and comments
### What changes were proposed in this pull request?
Fix typo for docs, log messages and comments

### Why are the changes needed?
typo fix to increase readability

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
manual test has been performed to test the updated

Closes #29443 from brandonJY/spell-fix-doc.

Authored-by: Brandon Jiang <Brandon.jiang.a@outlook.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-22 06:45:35 +09:00
Liang-Chi Hsieh e277ef1a83 [SPARK-32646][SQL] ORC predicate pushdown should work with case-insensitive analysis
### What changes were proposed in this pull request?

This PR proposes to fix ORC predicate pushdown under case-insensitive analysis case. The field names in pushed down predicates don't need to match in exact letter case with physical field names in ORC files, if we enable case-insensitive analysis.

### Why are the changes needed?

Currently ORC predicate pushdown doesn't work with case-insensitive analysis. A predicate "a < 0" cannot pushdown to ORC file with field name "A" under case-insensitive analysis.

But Parquet predicate pushdown works with this case. We should make ORC predicate pushdown work with case-insensitive analysis too.

### Does this PR introduce _any_ user-facing change?

Yes, after this PR, under case-insensitive analysis, ORC predicate pushdown will work.

### How was this patch tested?

Unit tests.

Closes #29457 from viirya/fix-orc-pushdown.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-21 07:57:24 +00:00