Commit graph

10267 commits

Author SHA1 Message Date
angerszhu 47326ac1c6
[SPARK-28704][SQL][TEST] Add back Skiped HiveExternalCatalogVersionsSuite in HiveSparkSubmitSuite at JDK9+
### What changes were proposed in this pull request?
We skip test HiveExternalCatalogVersionsSuite when testing with JAVA_9 or later because our previous version does not support JAVA_9 or later. We now add it back since we have a version supports JAVA_9 or later.

### Why are the changes needed?

To recover test coverage.

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

### How was this patch tested?
Check CI logs.

Closes #30428 from AngersZhuuuu/SPARK-28704.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-20 08:40:14 -08:00
ulysses 3384bda453 [SPARK-33468][SQL] ParseUrl in ANSI mode should fail if input string is not a valid url
### What changes were proposed in this pull request?

With `ParseUrl`, instead of return null we throw exception if input string is not a vaild url.

### Why are the changes needed?

For ANSI mode.

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

Yes, user will get exception if `set spark.sql.ansi.enabled=true`.

### How was this patch tested?

Add test.

Closes #30399 from ulysses-you/SPARK-33468.

Lead-authored-by: ulysses <youxiduo@weidian.com>
Co-authored-by: ulysses-you <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-20 13:23:08 +00:00
Max Gekk 870d409533 [SPARK-32512][SQL][TESTS][FOLLOWUP] Remove duplicate tests for ALTER TABLE .. PARTITIONS from DataSourceV2SQLSuite
### What changes were proposed in this pull request?
Remove tests from `DataSourceV2SQLSuite` that were copied to `AlterTablePartitionV2SQLSuite` by https://github.com/apache/spark/pull/29339.

### Why are the changes needed?
- To reduce tests execution time
- To improve test maintenance

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

### How was this patch tested?
By running the modified tests:
```
$ build/sbt "test:testOnly *DataSourceV2SQLSuite"
$ build/sbt "test:testOnly *AlterTablePartitionV2SQLSuite"
```

Closes #30444 from MaxGekk/dedup-tests-AlterTablePartitionV2SQLSuite.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-20 12:53:45 +00:00
Gabor Somogyi 883a213a8f
[MINOR] Structured Streaming statistics page indent fix
### What changes were proposed in this pull request?
Structured Streaming statistics page code contains an indentation issue. This PR fixes it.

### Why are the changes needed?
Indent fix.

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

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

Closes #30434 from gaborgsomogyi/STAT-INDENT-FIX.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-19 13:36:45 -08:00
Chao Sun 6da8ade5f4
[SPARK-33045][SQL][FOLLOWUP] Fix build failure with Scala 2.13
### What changes were proposed in this pull request?

Explicitly convert `scala.collection.mutable.Buffer` to `Seq`. In Scala 2.13 `Seq` is an alias of `scala.collection.immutable.Seq` instead of `scala.collection.Seq`.

### Why are the changes needed?

Without the change build with Scala 2.13 fails with the following:
```
[error] /home/runner/work/spark/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:1417:41: type mismatch;
[error]  found   : scala.collection.mutable.Buffer[org.apache.spark.unsafe.types.UTF8String]
[error]  required: Seq[org.apache.spark.unsafe.types.UTF8String]
[error]                 case null => LikeAll(e, patterns)
[error]                                         ^
[error] /home/runner/work/spark/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:1418:41: type mismatch;
[error]  found   : scala.collection.mutable.Buffer[org.apache.spark.unsafe.types.UTF8String]
[error]  required: Seq[org.apache.spark.unsafe.types.UTF8String]
[error]                 case _ => NotLikeAll(e, patterns)
[error]                                         ^
```

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

No

### How was this patch tested?

N/A

Closes #30431 from sunchao/SPARK-33045-followup.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-19 12:42:33 -08:00
gengjiaan 3695e997d5 [SPARK-33045][SQL] Support build-in function like_all and fix StackOverflowError issue
### What changes were proposed in this pull request?
Spark already support `LIKE ALL` syntax, but it will throw `StackOverflowError` if there are many elements(more than 14378 elements). We should implement built-in function for LIKE ALL to fix this issue.

Why the stack overflow can happen in the current approach ?
The current approach uses reduceLeft to connect each `Like(e, p)`, this will lead the the call depth of the thread is too large, causing `StackOverflowError` problems.

Why the fix in this PR can avoid the error?
This PR support built-in function for `LIKE ALL` and avoid this issue.

### Why are the changes needed?
1.Fix the `StackOverflowError` issue.
2.Support built-in function `like_all`.

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

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

Closes #29999 from beliefer/SPARK-33045-like_all.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-19 16:56:21 +00:00
ulysses 21b13506cd [SPARK-33442][SQL] Change Combine Limit to Eliminate limit using max row
### What changes were proposed in this pull request?

Change `CombineLimits` name to `EliminateLimits` and add check if `Limit` child max row <= limit.

### Why are the changes needed?

In Add-hoc scene, we always add limit for the query if user have no special limit value, but not all limit is nesessary.

A general negative example is
```
select count(*) from t limit 100000;
```

It will be great if we can eliminate limit at Spark side.

Also, we make a benchmark for this case
```
runBenchmark("Sort and Limit") {
  val N = 100000
  val benchmark = new Benchmark("benchmark sort and limit", N)

  benchmark.addCase("TakeOrderedAndProject", 3) { _ =>
    spark.range(N).toDF("c").repartition(200).sort("c").take(200000)
  }

  benchmark.addCase("Sort And Limit", 3) { _ =>
    withSQLConf("spark.sql.execution.topKSortFallbackThreshold" -> "-1") {
      spark.range(N).toDF("c").repartition(200).sort("c").take(200000)
    }
  }

  benchmark.addCase("Sort", 3) { _ =>
    spark.range(N).toDF("c").repartition(200).sort("c").collect()
  }
  benchmark.run()
}
```

and the result is
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_191-b12 on Mac OS X 10.15.6
Intel(R) Core(TM) i5-5257U CPU  2.70GHz
benchmark sort and limit:                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
TakeOrderedAndProject                              1833           2259         382          0.1       18327.1       1.0X
Sort And Limit                                     1417           1658         285          0.1       14167.5       1.3X
Sort                                               1324           1484         225          0.1       13238.3       1.4X
```

It shows that it makes sense to replace `TakeOrderedAndProjectExec` with `Sort + Project`.

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

No.

### How was this patch tested?

Add test.

Closes #30368 from ulysses-you/SPARK-33442.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-19 13:31:10 +00:00
allisonwang-db a03c540cf7 [SPARK-33472][SQL] Adjust RemoveRedundantSorts rule order
### What changes were proposed in this pull request?

This PR switched the order for the rule `RemoveRedundantSorts` and `EnsureRequirements` so that `EnsureRequirements` will be invoked before `RemoveRedundantSorts` to avoid IllegalArgumentException when instantiating PartitioningCollection.

### Why are the changes needed?
`RemoveRedundantSorts` rule uses SparkPlan's `outputPartitioning` to check whether a sort node is redundant. Currently, it is added before `EnsureRequirements`. Since `PartitioningCollection` requires left and right partitioning to have the same number of partitions, which is not necessarily true before applying `EnsureRequirements`, the rule can fail with the following exception:
```
IllegalArgumentException: requirement failed: PartitioningCollection requires all of its partitionings have the same numPartitions.
```

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

### How was this patch tested?
Unit test

Closes #30373 from allisonwang-db/sort-follow-up.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-19 13:29:01 +00:00
allisonwang-db ef2638c3e3
[SPARK-33183][SQL][FOLLOW-UP] Update rule RemoveRedundantSorts config version
### What changes were proposed in this pull request?
This PR is a follow up for #30093 to updates the config `spark.sql.execution.removeRedundantSorts` version to 2.4.8.

### Why are the changes needed?
To update the rule version it has been backported to 2.4. #30194

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

### How was this patch tested?
N/A

Closes #30420 from allisonwang-db/spark-33183-follow-up.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-19 00:12:22 -08:00
Dongjoon Hyun d5e7bd0cc4
[SPARK-33483][INFRA][TESTS] Fix rat exclusion patterns and add a LICENSE
### What changes were proposed in this pull request?

This PR fixes the RAT exclusion rule which was originated from SPARK-1144 (Apache Spark 1.0)

### Why are the changes needed?

This prevents the situation like https://github.com/apache/spark/pull/30415.

Currently, it missed `catalog` directory due to `.log` rule.
```
$ dev/check-license
Could not find Apache license headers in the following files:
 !????? /Users/dongjoon/APACHE/spark-merge/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/MetadataColumn.java
 !????? /Users/dongjoon/APACHE/spark-merge/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsMetadataColumns.java
```

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

No.

### How was this patch tested?

Pass the CI with the new rule.

Closes #30418 from dongjoon-hyun/SPARK-RAT.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-18 23:59:11 -08:00
Prakhar Jain 0b0fb70b09 [SPARK-33400][SQL] Normalize sameOrderExpressions in SortOrder to avoid unnecessary sort operations
### What changes were proposed in this pull request?
This pull request tries to normalize the SortOrder properly to prevent unnecessary sort operators. Currently the sameOrderExpressions are not normalized as part of AliasAwareOutputOrdering.

Example: consider this join of three tables:

      """
        |SELECT t2id, t3.id as t3id
        |FROM (
        |    SELECT t1.id as t1id, t2.id as t2id
        |    FROM t1, t2
        |    WHERE t1.id = t2.id
        |) t12, t3
        |WHERE t1id = t3.id
      """.

The plan for this looks like:

      *(8) Project [t2id#1059L, id#1004L AS t3id#1060L]
      +- *(8) SortMergeJoin [t2id#1059L], [id#1004L], Inner
         :- *(5) Sort [t2id#1059L ASC NULLS FIRST ], false, 0         <-----------------------------
         :  +- *(5) Project [id#1000L AS t2id#1059L]
         :     +- *(5) SortMergeJoin [id#996L], [id#1000L], Inner
         :        :- *(2) Sort [id#996L ASC NULLS FIRST ], false, 0
         :        :  +- Exchange hashpartitioning(id#996L, 5), true, [id=#1426]
         :        :     +- *(1) Range (0, 10, step=1, splits=2)
         :        +- *(4) Sort [id#1000L ASC NULLS FIRST ], false, 0
         :           +- Exchange hashpartitioning(id#1000L, 5), true, [id=#1432]
         :              +- *(3) Range (0, 20, step=1, splits=2)
         +- *(7) Sort [id#1004L ASC NULLS FIRST ], false, 0
            +- Exchange hashpartitioning(id#1004L, 5), true, [id=#1443]
               +- *(6) Range (0, 30, step=1, splits=2)

In this plan, the marked sort node could have been avoided as the data is already sorted on "t2.id" by the lower SortMergeJoin.

### Why are the changes needed?
To remove unneeded Sort operators.

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

### How was this patch tested?
New UT added.

Closes #30302 from prakharjain09/SPARK-33400-sortorder.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-19 06:25:37 +00:00
Yuming Wang 014e1fbb3a [SPARK-27421][SQL] Fix filter for int column and value class java.lang.String when pruning partition column
### What changes were proposed in this pull request?

This pr fix filter for int column and value class java.lang.String when pruning partition column.

How to reproduce this issue:
```scala
spark.sql("CREATE table test (name STRING) partitioned by (id int) STORED AS PARQUET")
spark.sql("CREATE VIEW test_view as select cast(id as string) as id, name from test")
spark.sql("SELECT * FROM test_view WHERE id = '0'").explain
```
```
20/11/15 06:19:01 INFO audit: ugi=root ip=unknown-ip-addr cmd=get_partitions_by_filter : db=default tbl=test
20/11/15 06:19:01 INFO MetaStoreDirectSql: Unable to push down SQL filter: Cannot push down filter for int column and value class java.lang.String
20/11/15 06:19:01 ERROR SparkSQLDriver: Failed in [SELECT * FROM test_view WHERE id = '0']
java.lang.RuntimeException: Caught Hive MetaException attempting to get partition metadata by filter from Hive. You can set the Spark configuration setting spark.sql.hive.manageFilesourcePartitions to false to work around this problem, however this will result in degraded performance. Please report a bug: https://issues.apache.org/jira/browse/SPARK
 at org.apache.spark.sql.hive.client.Shim_v0_13.getPartitionsByFilter(HiveShim.scala:828)
 at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getPartitionsByFilter$1(HiveClientImpl.scala:745)
 at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:294)
 at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:227)
 at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:226)
 at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:276)
 at org.apache.spark.sql.hive.client.HiveClientImpl.getPartitionsByFilter(HiveClientImpl.scala:743)
```

### Why are the changes needed?

Fix bug.

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

No.

### How was this patch tested?

Unit test.

Closes #30380 from wangyum/SPARK-27421.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2020-11-19 14:01:42 +08:00
yangjie01 e3058ba17c [SPARK-33441][BUILD] Add unused-imports compilation check and remove all unused-imports
### What changes were proposed in this pull request?
This pr add a new Scala compile arg to `pom.xml` to defense against new unused imports:

- `-Ywarn-unused-import` for Scala 2.12
- `-Wconf:cat=unused-imports:e` for Scala 2.13

The other fIles change are remove all unused imports in Spark code

### Why are the changes needed?
Cleanup code and add guarantee to defense against new unused imports

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #30351 from LuciferYang/remove-imports-core-module.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-19 14:20:39 +09:00
Ryan Blue 66a76378cf
[SPARK-31255][SQL][FOLLOWUP] Add missing license headers
### What changes were proposed in this pull request?

Add missing license headers for new files added in #28027.

### Why are the changes needed?

To fix licenses.

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

No.

### How was this patch tested?

This is a purely non-functional change.

Closes #30415 from rdblue/license-headers.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-18 19:18:28 -08:00
Liang-Chi Hsieh e518008ca9
[SPARK-33473][SQL] Extend interpreted subexpression elimination to other interpreted projections
### What changes were proposed in this pull request?

Similar to `InterpretedUnsafeProjection`, this patch proposes to extend interpreted subexpression elimination to `InterpretedMutableProjection` and `InterpretedSafeProjection`.

### Why are the changes needed?

Enabling subexpression elimination can improve the performance of interpreted projections, as shown in `InterpretedUnsafeProjection`.

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

No

### How was this patch tested?

Unit test.

Closes #30406 from viirya/SPARK-33473.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-18 18:58:06 -08:00
Liang-Chi Hsieh 97d2cee4af [SPARK-33427][SQL][FOLLOWUP] Prevent test flakyness in SubExprEvaluationRuntimeSuite
### What changes were proposed in this pull request?

This followup is to prevent possible test flakyness of `SubExprEvaluationRuntimeSuite`.

### Why are the changes needed?

Because HashMap doesn't guarantee the order, in `proxyExpressions` the proxy expression id is not deterministic. So in `SubExprEvaluationRuntimeSuite` we should not test against it.

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

No, dev only.

### How was this patch tested?

Unit test.

Closes #30414 from viirya/SPARK-33427-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-11-18 18:35:11 -08:00
Gengliang Wang 9a4c79073b [SPARK-33354][SQL] New explicit cast syntax rules in ANSI mode
### What changes were proposed in this pull request?

In section 6.13 of the ANSI SQL standard, there are syntax rules for valid combinations of the source and target data types.
![image](https://user-images.githubusercontent.com/1097932/98212874-17356f80-1ef9-11eb-8f2b-385f32db404a.png)

Comparing the ANSI CAST syntax rules with the current default behavior of Spark:
![image](https://user-images.githubusercontent.com/1097932/98789831-b7870a80-23b7-11eb-9b5f-469a42e0ee4a.png)

To make Spark's ANSI mode more ANSI SQL Compatible,I propose to disallow the following casting in ANSI mode:
```
TimeStamp <=> Boolean
Date <=> Boolean
Numeric <=> Timestamp
Numeric <=> Date
Numeric <=> Binary
String <=> Array
String <=> Map
String <=> Struct
```
The following castings are considered invalid in ANSI SQL standard, but they are quite straight forward. Let's Allow them for now
```
Numeric <=> Boolean
String <=> Binary
```
### Why are the changes needed?

Better ANSI SQL compliance

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

Yes, the following castings will not be allowed in ANSI mode:
```
TimeStamp <=> Boolean
Date <=> Boolean
Numeric <=> Timestamp
Numeric <=> Date
Numeric <=> Binary
String <=> Array
String <=> Map
String <=> Struct
```

### How was this patch tested?

Unit test

The ANSI Compliance doc preview:
![image](https://user-images.githubusercontent.com/1097932/98946017-2cd20880-24a8-11eb-8161-65749bfdd03a.png)

Closes #30260 from gengliangwang/ansiCanCast.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-11-19 09:23:36 +09:00
Ryan Blue 1df69f7e32 [SPARK-31255][SQL] Add SupportsMetadataColumns to DSv2
### What changes were proposed in this pull request?

This adds support for metadata columns to DataSourceV2. If a source implements `SupportsMetadataColumns` it must also implement `SupportsPushDownRequiredColumns` to support projecting those columns.

The analyzer is updated to resolve metadata columns from `LogicalPlan.metadataOutput`, and this adds a rule that will add metadata columns to the output of `DataSourceV2Relation` if one is used.

### Why are the changes needed?

This is the solution discussed for exposing additional data in the Kafka source. It is also needed for a generic `MERGE INTO` plan.

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

Yes. Users can project additional columns from sources that implement the new API. This also updates `DescribeTableExec` to show metadata columns.

### How was this patch tested?

Will include new unit tests.

Closes #28027 from rdblue/add-dsv2-metadata-columns.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2020-11-18 14:07:51 -08:00
Chao Sun 27cd945c15 [SPARK-32381][CORE][SQL][FOLLOWUP] More cleanup on HadoopFSUtils
### What changes were proposed in this pull request?

This PR is a follow-up of #29471 and does the following improvements for `HadoopFSUtils`:
1. Removes the extra `filterFun` from the listing API and combines it with the `filter`.
2. Removes `SerializableBlockLocation` and `SerializableFileStatus` given that `BlockLocation` and `FileStatus` are already serializable.
3. Hides the `isRootLevel` flag from the top-level API.

### Why are the changes needed?

Main purpose is to simplify the logic within `HadoopFSUtils` as well as cleanup the API.

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

No

### How was this patch tested?

Existing unit tests (e.g., `FileIndexSuite`)

Closes #29959 from sunchao/hadoop-fs-utils-followup.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-11-18 12:39:00 -08:00
Gengliang Wang a180e02842 [SPARK-32852][SQL][DOC][FOLLOWUP] Revise the documentation of spark.sql.hive.metastore.jars
### What changes were proposed in this pull request?

This is a follow-up for https://github.com/apache/spark/pull/29881.
It revises the documentation of the configuration `spark.sql.hive.metastore.jars`.

### Why are the changes needed?

Fix grammatical error in the doc.
Also, make it more clear that the configuration is effective only when `spark.sql.hive.metastore.jars` is set as `path`

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

No

### How was this patch tested?

Just doc changes.

Closes #30407 from gengliangwang/reviseJarPathDoc.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-11-18 22:09:40 +08:00
Bryan Cutler 8e2a0bdce7 [SPARK-24554][PYTHON][SQL] Add MapType support for PySpark with Arrow
### What changes were proposed in this pull request?

This change adds MapType support for PySpark with Arrow, if using pyarrow >= 2.0.0.

### Why are the changes needed?

MapType was previous unsupported with Arrow.

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

User can now enable MapType for `createDataFrame()`, `toPandas()` with Arrow optimization, and with Pandas UDFs.

### How was this patch tested?

Added new PySpark tests for createDataFrame(), toPandas() and Scalar Pandas UDFs.

Closes #30393 from BryanCutler/arrow-add-MapType-SPARK-24554.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-18 21:18:19 +09:00
Liang-Chi Hsieh 7f3d99a8a5 [MINOR][SQL][DOCS] Update schema_of_csv and schema_of_json doc
### What changes were proposed in this pull request?

This minor PR updates the docs of `schema_of_csv` and `schema_of_json`. They allow foldable string column instead of a string literal now.

### Why are the changes needed?

The function doc of  `schema_of_csv` and `schema_of_json` are not updated accordingly with previous PRs.

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

Yes, update user-facing doc.

### How was this patch tested?

Unit test.

Closes #30396 from viirya/minor-json-csv.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-18 11:32:27 +09:00
Liang-Chi Hsieh 928348408e [SPARK-33427][SQL] Add subexpression elimination for interpreted expression evaluation
### What changes were proposed in this pull request?

This patch proposes to add subexpression elimination for interpreted expression evaluation. Interpreted expression evaluation is used when codegen was not able to work, for example complex schema.

### Why are the changes needed?

Currently we only do subexpression elimination for codegen. For some reasons, we may need to run interpreted expression evaluation. For example, codegen fails to compile and fallbacks to interpreted mode, or complex input/output schema of expressions. It is commonly seen for complex schema from expressions that is possibly caused by the query optimizer too, e.g. SPARK-32945.

We should also support subexpression elimination for interpreted evaluation. That could reduce performance difference when Spark fallbacks from codegen to interpreted expression evaluation, and improve Spark usability.

#### Benchmark

Update `SubExprEliminationBenchmark`:

Before:

```
OpenJDK 64-Bit Server VM 1.8.0_265-b01 on Mac OS X 10.15.6
 Intel(R) Core(TM) i7-9750H CPU  2.60GHz
 from_json as subExpr:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 -------------------------------------------------------------------------------------------------------------------------
subexpressionElimination on, codegen off           24707          25688         903          0.0   247068775.9       1.0X
```

After:
```
OpenJDK 64-Bit Server VM 1.8.0_265-b01 on Mac OS X 10.15.6
 Intel(R) Core(TM) i7-9750H CPU  2.60GHz
 from_json as subExpr:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 -------------------------------------------------------------------------------------------------------------------------
subexpressionElimination on, codegen off            2360           2435          87          0.0    23604320.7      11.2X
```

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

No

### How was this patch tested?

Unit test. Benchmark manually.

Closes #30341 from viirya/SPARK-33427.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-17 14:29:37 +00:00
Yuming Wang 09bb9bedcd [SPARK-33416][SQL] Avoid Hive metastore stack overflow when InSet predicate have many values
### What changes were proposed in this pull request?

We [rewrite](5197c5d2e7/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala (L722-L724)) `In`/`InSet` predicate to `or` expressions when pruning Hive partitions. That will cause Hive metastore stack over flow if there are a lot of values.

This pr rewrite `InSet` predicate to `GreaterThanOrEqual` min value and `LessThanOrEqual ` max value when pruning Hive partitions to avoid Hive metastore stack overflow.

From our experience, `spark.sql.hive.metastorePartitionPruningInSetThreshold` should be less than 10000.

### Why are the changes needed?

Avoid Hive metastore stack overflow when `InSet` predicate have many values.
Especially DPP, it may generate many values.

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

No.

### How was this patch tested?

Manual test.

Closes #30325 from wangyum/SPARK-33416.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-17 13:47:01 +00:00
HyukjinKwon e2c7bfce40 [SPARK-33407][PYTHON] Simplify the exception message from Python UDFs (disabled by default)
### What changes were proposed in this pull request?

This PR proposes to simplify the exception messages from Python UDFS.

Currently, the exception message from Python UDFs is as below:

```python
from pyspark.sql.functions import udf; spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```

```python
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../python/pyspark/sql/dataframe.py", line 427, in show
    print(self._jdf.showString(n, 20, vertical))
  File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
  File "/.../python/pyspark/sql/utils.py", line 127, in deco
    raise_from(converted)
  File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
  An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
  File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
    process()
  File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
    serializer.dump_stream(out_iter, outfile)
  File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
    self.serializer.dump_stream(self._batched(iterator), stream)
  File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
    for obj in iterator:
  File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
    for item in iterator:
  File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
    return lambda *a: f(*a)
  File "/.../python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
    return f(*args, **kwargs)
  File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```

Actually, almost all cases, users only care about `ZeroDivisionError: division by zero`. We don't really have to show the internal stuff in 99% cases.

This PR adds a configuration `spark.sql.execution.pyspark.udf.simplifiedException.enabled` (disabled by default) that hides the internal tracebacks related to Python worker, (de)serialization, etc.

```python
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../python/pyspark/sql/dataframe.py", line 427, in show
    print(self._jdf.showString(n, 20, vertical))
  File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
  File "/.../python/pyspark/sql/utils.py", line 127, in deco
    raise_from(converted)
  File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
  An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
  File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```

The trackback will be shown from the point when any non-PySpark file is seen in the traceback.

### Why are the changes needed?

Without this configuration. such internal tracebacks are exposed to users directly especially for shall or notebook users in PySpark. 99% cases people don't care about the internal Python worker, (de)serialization and related tracebacks. It just makes the exception more difficult to read. For example, one statement of `x/0` above shows a very long traceback and most of them are unnecessary.

This configuration enables the ability to show simplified tracebacks which users will likely be most interested in.

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

By default, no. It adds one configuration that simplifies the exception message. See the example above.

### How was this patch tested?

Manually tested:

```bash
$ pyspark --conf spark.sql.execution.pyspark.udf.simplifiedException.enabled=true
```
```python
from pyspark.sql.functions import udf; spark.sparkContext.setLogLevel("FATAL"); spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```

and unittests were also added.

Closes #30309 from HyukjinKwon/SPARK-33407.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-17 14:15:31 +09:00
Cheng Su 5af5aa146e [SPARK-33209][SS] Refactor unit test of stream-stream join in UnsupportedOperationsSuite
### What changes were proposed in this pull request?

This PR is a followup from https://github.com/apache/spark/pull/30076 to refactor unit test of stream-stream join in `UnsupportedOperationsSuite`, where we had a lot of duplicated code for stream-stream join unit test, for each join type.

### Why are the changes needed?

Help reduce duplicated code and make it easier for developers to read and add code in the future.

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

No.

### How was this patch tested?

Existing unit test in `UnsupportedOperationsSuite.scala` (pure refactoring).

Closes #30347 from c21/stream-test.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-17 11:18:42 +09:00
Prakhar Jain f5e3302840 [SPARK-33399][SQL] Normalize output partitioning and sortorder with respect to aliases to avoid unneeded exchange/sort nodes
### What changes were proposed in this pull request?
This pull request tries to remove unneeded exchanges/sorts by normalizing the output partitioning and sortorder information correctly with respect to aliases.

Example: consider this join of three tables:

     |SELECT t2id, t3.id as t3id
     |FROM (
     |    SELECT t1.id as t1id, t2.id as t2id
     |    FROM t1, t2
     |    WHERE t1.id = t2.id
     |) t12, t3
     |WHERE t1id = t3.id

The plan for this looks like:

      *(9) Project [t2id#1034L, id#1004L AS t3id#1035L]
      +- *(9) SortMergeJoin [t1id#1033L], [id#1004L], Inner
         :- *(6) Sort [t1id#1033L ASC NULLS FIRST], false, 0
         :  +- Exchange hashpartitioning(t1id#1033L, 5), true, [id=#1343]   <------------------------------
         :     +- *(5) Project [id#996L AS t1id#1033L, id#1000L AS t2id#1034L]
         :        +- *(5) SortMergeJoin [id#996L], [id#1000L], Inner
         :           :- *(2) Sort [id#996L ASC NULLS FIRST], false, 0
         :           :  +- Exchange hashpartitioning(id#996L, 5), true, [id=#1329]
         :           :     +- *(1) Range (0, 10, step=1, splits=2)
         :           +- *(4) Sort [id#1000L ASC NULLS FIRST], false, 0
         :              +- Exchange hashpartitioning(id#1000L, 5), true, [id=#1335]
         :                 +- *(3) Range (0, 20, step=1, splits=2)
         +- *(8) Sort [id#1004L ASC NULLS FIRST], false, 0
            +- Exchange hashpartitioning(id#1004L, 5), true, [id=#1349]
               +- *(7) Range (0, 30, step=1, splits=2)

In this plan, the marked exchange could have been avoided as the data is already partitioned on "t1.id". This happens because AliasAwareOutputPartitioning class handles aliases only related to HashPartitioning. This change normalizes all output partitioning based on aliasing happening in Project.

### Why are the changes needed?
To remove unneeded exchanges.

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

### How was this patch tested?
New UT added.

On TPCDS 1000 scale, this change improves the performance of query 95 from 330 seconds to 170 seconds by removing the extra Exchange.

Closes #30300 from prakharjain09/SPARK-33399-outputpartitioning.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-11-17 10:35:43 +09:00
xuewei.linxuewei b5eca18af0 [SPARK-33460][SQL] Accessing map values should fail if key is not found
### What changes were proposed in this pull request?

Instead of returning NULL, throws runtime NoSuchElementException towards invalid key accessing in map-like functions, such as element_at, GetMapValue, when ANSI mode is on.

### Why are the changes needed?

For ANSI mode.

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

No.

### How was this patch tested?

Added UT and Existing UT.

Closes #30386 from leanken/leanken-SPARK-33460.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-16 16:14:31 +00:00
Max Gekk 6883f29465 [SPARK-33453][SQL][TESTS] Unify v1 and v2 SHOW PARTITIONS tests
### What changes were proposed in this pull request?
1. Move `SHOW PARTITIONS` parsing tests to `ShowPartitionsParserSuite`
2. Place Hive tests for `SHOW PARTITIONS` from `HiveCommandSuite` to the base test suite `v1.ShowPartitionsSuiteBase`. This will allow to run the tests w/ and w/o Hive.

The changes follow the approach of https://github.com/apache/spark/pull/30287.

### Why are the changes needed?
- The unification will allow to run common `SHOW PARTITIONS` tests for both DSv1 and Hive DSv1, DSv2
- We can detect missing features and differences between DSv1 and DSv2 implementations.

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

### How was this patch tested?
By running:
- new test suites `build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowPartitionsSuite"`
- and old one `build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly org.apache.spark.sql.hive.execution.HiveCommandSuite"`

Closes #30377 from MaxGekk/unify-dsv1_v2-show-partitions-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-16 16:11:42 +00:00
luluorta dfa6fb46f4 [SPARK-33389][SQL] Make internal classes of SparkSession always using active SQLConf
### What changes were proposed in this pull request?

This PR makes internal classes of SparkSession always using active SQLConf. We should remove all `conf: SQLConf`s from ctor-parameters of this classes (`Analyzer`, `SparkPlanner`, `SessionCatalog`, `CatalogManager` `SparkSqlParser` and etc.) and use  `SQLConf.get` instead.

### Why are the changes needed?

Code refine.

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

No.

### How was this patch tested?

Existing test

Closes #30299 from luluorta/SPARK-33389.

Authored-by: luluorta <luluorta@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-16 15:27:18 +00:00
xuewei.linxuewei aa508fcc03 [SPARK-33140][SQL][FOLLOW-UP] Revert code that not use passed-in SparkSession to get SQLConf
### What changes were proposed in this pull request?

Revert code that does not use passed-in SparkSession to get SQLConf in [SPARK-33140]. The change scope of [SPARK-33140] change passed-in SQLConf instance and place using SparkSession to get SQLConf to be unified to use SQLConf.get. And the code reverted in the patch, the passed-in SparkSession was not about to get SQLConf, but using its catalog, it's better to be consistent.

### Why are the changes needed?

Potential regression bug.

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

No.

### How was this patch tested?

Existing UT.

Closes #30364 from leanken/leanken-SPARK-33140.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-16 11:57:50 +00:00
Max Gekk 71a29b2eca [MINOR][SQL][DOCS] Fix a reference to spark.sql.sources.useV1SourceList
### What changes were proposed in this pull request?
Replace `spark.sql.sources.write.useV1SourceList` by `spark.sql.sources.useV1SourceList` in the comment for `CatalogManager.v2SessionCatalog()`.

### Why are the changes needed?
To have correct comments.

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

### How was this patch tested?
By running `./dev/scalastyle`.

Closes #30385 from MaxGekk/fix-comment-useV1SourceList.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-16 17:57:20 +09:00
Liang-Chi Hsieh 10b011f837 [SPARK-33456][SQL][TEST][FOLLOWUP] Fix SUBEXPRESSION_ELIMINATION_ENABLED config name
### What changes were proposed in this pull request?

To fix wrong config name in `subexp-elimination.sql`.

### Why are the changes needed?

`CONFIG_DIM` should use config name's key.

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

No, dev only.

### How was this patch tested?

Unit test.

Closes #30384 from viirya/SPARK-33456-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-16 17:53:31 +09:00
Yuming Wang cdcbdaeb0d [SPARK-33458][SQL] Hive partition pruning support Contains, StartsWith and EndsWith predicate
### What changes were proposed in this pull request?

This pr add support Hive partition pruning on `Contains`, `StartsWith` and `EndsWith` predicate.

### Why are the changes needed?

Improve query performance.

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

No.

### How was this patch tested?

Unit test.

Closes #30383 from wangyum/SPARK-33458.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-16 07:18:13 +00:00
Max Gekk 4e5d2e0695 [SPARK-33394][SQL][TESTS] Throw NoSuchNamespaceException for not existing namespace in InMemoryTableCatalog.listTables()
### What changes were proposed in this pull request?
Throw `NoSuchNamespaceException` in `listTables()` of the custom test catalog `InMemoryTableCatalog` if the passed namespace doesn't exist.

### Why are the changes needed?
1. To align behavior of V2 `InMemoryTableCatalog` to V1 session catalog.
2. To distinguish two situations:
    1. A namespace **does exist** but does not contain any tables. In that case, `listTables()` returns empty result.
    2. A namespace **does not exist**. `listTables()` throws `NoSuchNamespaceException` in this case.

### Does this PR introduce _any_ user-facing change?
Yes. For example, `SHOW TABLES` returns empty result before the changes.

### How was this patch tested?
By running V1/V2 ShowTablesSuites.

Closes #30358 from MaxGekk/show-tables-in-not-existing-namespace.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-16 07:08:21 +00:00
Liang-Chi Hsieh d4cf1483fd [SPARK-33456][SQL][TEST] Add end-to-end test for subexpression elimination
### What changes were proposed in this pull request?

This patch proposes to add end-to-end test for subexpression elimination.

### Why are the changes needed?

We have subexpression elimination feature for expression evaluation but we don't have end-to-end tests for the feature. We should have one to make sure we don't break it.

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

No, dev only.

### How was this patch tested?

Unit tests.

Closes #30381 from viirya/SPARK-33456.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-16 15:47:35 +09:00
artiship 1ae6d64b5f
[SPARK-33358][SQL] Return code when command process failed
Exit Spark SQL CLI processing loop if one of the commands (sub sql statement) process failed

This is a regression at Apache Spark 3.0.0.

```
$ cat 1.sql
select * from nonexistent_table;
select 2;
```

**Apache Spark 2.4.7**
```
spark-2.4.7-bin-hadoop2.7:$ bin/spark-sql -f 1.sql
20/11/15 16:14:38 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Error in query: Table or view not found: nonexistent_table; line 1 pos 14
```

**Apache Spark 3.0.1**
```
$ bin/spark-sql -f 1.sql
Error in query: Table or view not found: nonexistent_table; line 1 pos 14;
'Project [*]
+- 'UnresolvedRelation [nonexistent_table]

2
Time taken: 2.786 seconds, Fetched 1 row(s)
```

**Apache Hive 1.2.2**
```
apache-hive-1.2.2-bin:$ bin/hive -f 1.sql

Logging initialized using configuration in jar:file:/Users/dongjoon/APACHE/hive-release/apache-hive-1.2.2-bin/lib/hive-common-1.2.2.jar!/hive-log4j.properties
FAILED: SemanticException [Error 10001]: Line 1:14 Table not found 'nonexistent_table'
```

Yes. This is a fix of regression.

Pass the UT.

Closes #30263 from artiship/SPARK-33358.

Authored-by: artiship <meilziner@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-15 16:57:12 -08:00
Liang-Chi Hsieh eea846b895
[SPARK-33455][SQL][TEST] Add SubExprEliminationBenchmark for benchmarking subexpression elimination
### What changes were proposed in this pull request?

This patch adds a benchmark `SubExprEliminationBenchmark` for benchmarking subexpression elimination feature.

### Why are the changes needed?

We need a benchmark for subexpression elimination feature for change such as #30341.

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

No, dev only.

### How was this patch tested?

Unit test.

Closes #30379 from viirya/SPARK-33455.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-14 19:02:36 -08:00
luluorta 156704ba0d
[SPARK-33432][SQL] SQL parser should use active SQLConf
### What changes were proposed in this pull request?

This PR makes SQL parser using active SQLConf instead of the one in ctor-parameters.

### Why are the changes needed?

In ANSI mode, schema string parsing should fail if the schema uses ANSI reserved keyword as attribute name:

```scala
spark.conf.set("spark.sql.ansi.enabled", "true")
spark.sql("""select from_json('{"time":"26/10/2015"}', 'time Timestamp', map('timestampFormat',  'dd/MM/yyyy'));""").show
```

output:

> Cannot parse the data type:
> no viable alternative at input 'time'(line 1, pos 0)
>
> == SQL ==
> time Timestamp
> ^^^

But this query may accidentally succeed in certain cases cause the DataType parser sticks to the configs of the first created session in the current thread:

```scala
DataType.fromDDL("time Timestamp")
val newSpark = spark.newSession()
newSpark.conf.set("spark.sql.ansi.enabled", "true")
newSpark.sql("""select from_json('{"time":"26/10/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy'));""").show
```

output:

> +--------------------------------+
> |from_json({"time":"26/10/2015"})|
> +--------------------------------+
> |                   {2015-10-26 00:00...|
> +--------------------------------+

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

No.

### How was this patch tested?

Newly and updated UTs

Closes #30357 from luluorta/SPARK-33432.

Authored-by: luluorta <luluorta@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-14 13:37:12 -08:00
artiship 34a9a77ab5 [SPARK-33396][SQL] Spark SQL CLI prints appliction id when process file
### What changes were proposed in this pull request?
Modify SparkSQLCLIDriver.scala to move ahead calling the cli.printMasterAndAppId method before process file.

### Why are the changes needed?
Even though in SPARK-25043 it has already brought in the printing application id feature. But the process file situation seems have not been included. This small change is to make spark-sql will also print out application id when process file.

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

### How was this patch tested?
env

```
spark version: 3.0.1
os: centos 7
```

/tmp/tmp.sql

```sql
select 1;
```

submit command:

```sh
export HADOOP_USER_NAME=my-hadoop-user
bin/spark-sql  \
--master yarn \
--deploy-mode client \
--queue my.queue.name \
--conf spark.driver.host=$(hostname -i) \
--conf spark.app.name=spark-test  \
--name "spark-test" \
-f /tmp/tmp.sql
```

execution log:

```sh
20/11/09 23:18:39 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
20/11/09 23:18:40 WARN HiveConf: HiveConf of name hive.spark.client.rpc.server.address.use.ip does not exist
20/11/09 23:18:40 WARN HiveConf: HiveConf of name hive.spark.client.submit.timeout.interval does not exist
20/11/09 23:18:40 WARN HiveConf: HiveConf of name hive.enforce.bucketing does not exist
20/11/09 23:18:40 WARN HiveConf: HiveConf of name hive.server2.enable.impersonation does not exist
20/11/09 23:18:40 WARN HiveConf: HiveConf of name hive.run.timeout.seconds does not exist
20/11/09 23:18:40 WARN HiveConf: HiveConf of name hive.support.sql11.reserved.keywords does not exist
20/11/09 23:18:40 WARN DomainSocketFactory: The short-circuit local reads feature cannot be used because libhadoop cannot be loaded.
20/11/09 23:18:41 WARN SparkConf: Note that spark.local.dir will be overridden by the value set by the cluster manager (via SPARK_LOCAL_DIRS in mesos/standalone/kubernetes and LOCAL_DIRS in YARN).
20/11/09 23:18:42 WARN Client: Neither spark.yarn.jars nor spark.yarn.archive is set, falling back to uploading libraries under SPARK_HOME.
20/11/09 23:18:52 WARN YarnSchedulerBackend$YarnSchedulerEndpoint: Attempted to request executors before the AM has registered!

Spark master: yarn, Application Id: application_1567136266901_27355775
1
1
Time taken: 4.974 seconds, Fetched 1 row(s)

```

Closes #30301 from artiship/SPARK-33396.

Authored-by: artiship <meilziner@gmail.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2020-11-14 20:54:17 +08:00
Liang-Chi Hsieh 0046222a75
[SPARK-33337][SQL][FOLLOWUP] Prevent possible flakyness in SubexpressionEliminationSuite
### What changes were proposed in this pull request?

This is a simple followup to prevent test flakyness in SubexpressionEliminationSuite. If `getAllEquivalentExprs` returns more than 1 sequences, due to HashMap, we should use `contains` instead of assuming the order of results.

### Why are the changes needed?

Prevent test flakyness in SubexpressionEliminationSuite.

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

No, dev only.

### How was this patch tested?

Unit test.

Closes #30371 from viirya/SPARK-33337-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-13 15:10:02 -08:00
xuewei.linxuewei 234711a328 Revert "[SPARK-33139][SQL] protect setActionSession and clearActiveSession"
### What changes were proposed in this pull request?

In [SPARK-33139] we defined `setActionSession` and `clearActiveSession` as deprecated API, it turns out it is widely used, and after discussion, even if without this PR, it should work with unify view feature, it might only be a risk if user really abuse using these two API. So revert the PR is needed.

[SPARK-33139] has two commit, include a follow up. Revert them both.

### Why are the changes needed?

Revert.

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

No.

### How was this patch tested?

Existing UT.

Closes #30367 from leanken/leanken-revert-SPARK-33139.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-13 13:35:45 +00:00
Kent Yao cdd8e51742 [SPARK-33419][SQL] Unexpected behavior when using SET commands before a query in SparkSession.sql
### What changes were proposed in this pull request?

SparkSession.sql converts a string value to a DataFrame, and the string value should be one single SQL statement ending up w/ or w/o one or more semicolons. e.g.

```sql
scala> spark.sql(" select 2").show
+---+
|  2|
+---+
|  2|
+---+
scala> spark.sql(" select 2;").show
+---+
|  2|
+---+
|  2|
+---+

scala> spark.sql(" select 2;;;;").show
+---+
|  2|
+---+
|  2|
+---+
```
If we put 2 or more statements in, it fails in the parser as expected, e.g.

```sql
scala> spark.sql(" select 2; select 1;").show
org.apache.spark.sql.catalyst.parser.ParseException:
extraneous input 'select' expecting {<EOF>, ';'}(line 1, pos 11)

== SQL ==
 select 2; select 1;
-----------^^^

  at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:263)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:130)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:51)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:81)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$2(SparkSession.scala:610)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:610)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:769)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:607)
  ... 47 elided
```

As a very generic user scenario, users may want to change some settings before they execute
the queries. They may pass a string value like `set spark.sql.abc=2; select 1;` into this API, which creates a confusing gap between the actual effect and the user's expectations.

The user may want the query to be executed with spark.sql.abc=2, but Spark actually treats the whole part of `2; select 1;` as the value of the property 'spark.sql.abc',
e.g.

```
scala> spark.sql("set spark.sql.abc=2; select 1;").show
+-------------+------------+
|          key|       value|
+-------------+------------+
|spark.sql.abc|2; select 1;|
+-------------+------------+
```

What's more, the SET symbol could digest everything behind it, which makes it unstable from version to version, e.g.

#### 3.1
```sql
scala> spark.sql("set;").show
org.apache.spark.sql.catalyst.parser.ParseException:
Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, please use quotes, e.g., SET `ke y`=value.(line 1, pos 0)

== SQL ==
set;
^^^

  at org.apache.spark.sql.execution.SparkSqlAstBuilder.$anonfun$visitSetConfiguration$1(SparkSqlParser.scala:83)
  at org.apache.spark.sql.catalyst.parser.ParserUtils$.withOrigin(ParserUtils.scala:113)
  at org.apache.spark.sql.execution.SparkSqlAstBuilder.visitSetConfiguration(SparkSqlParser.scala:72)
  at org.apache.spark.sql.execution.SparkSqlAstBuilder.visitSetConfiguration(SparkSqlParser.scala:58)
  at org.apache.spark.sql.catalyst.parser.SqlBaseParser$SetConfigurationContext.accept(SqlBaseParser.java:2161)
  at org.antlr.v4.runtime.tree.AbstractParseTreeVisitor.visit(AbstractParseTreeVisitor.java:18)
  at org.apache.spark.sql.catalyst.parser.AstBuilder.$anonfun$visitSingleStatement$1(AstBuilder.scala:77)
  at org.apache.spark.sql.catalyst.parser.ParserUtils$.withOrigin(ParserUtils.scala:113)
  at org.apache.spark.sql.catalyst.parser.AstBuilder.visitSingleStatement(AstBuilder.scala:77)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.$anonfun$parsePlan$1(ParseDriver.scala:82)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:113)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:51)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:81)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$2(SparkSession.scala:610)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:610)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:769)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:607)
  ... 47 elided

scala> spark.sql("set a;").show
org.apache.spark.sql.catalyst.parser.ParseException:
Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, please use quotes, e.g., SET `ke y`=value.(line 1, pos 0)

== SQL ==
set a;
^^^

  at org.apache.spark.sql.execution.SparkSqlAstBuilder.$anonfun$visitSetConfiguration$1(SparkSqlParser.scala:83)
  at org.apache.spark.sql.catalyst.parser.ParserUtils$.withOrigin(ParserUtils.scala:113)
  at org.apache.spark.sql.execution.SparkSqlAstBuilder.visitSetConfiguration(SparkSqlParser.scala:72)
  at org.apache.spark.sql.execution.SparkSqlAstBuilder.visitSetConfiguration(SparkSqlParser.scala:58)
  at org.apache.spark.sql.catalyst.parser.SqlBaseParser$SetConfigurationContext.accept(SqlBaseParser.java:2161)
  at org.antlr.v4.runtime.tree.AbstractParseTreeVisitor.visit(AbstractParseTreeVisitor.java:18)
  at org.apache.spark.sql.catalyst.parser.AstBuilder.$anonfun$visitSingleStatement$1(AstBuilder.scala:77)
  at org.apache.spark.sql.catalyst.parser.ParserUtils$.withOrigin(ParserUtils.scala:113)
  at org.apache.spark.sql.catalyst.parser.AstBuilder.visitSingleStatement(AstBuilder.scala:77)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.$anonfun$parsePlan$1(ParseDriver.scala:82)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:113)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:51)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:81)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$2(SparkSession.scala:610)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:610)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:769)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:607)
  ... 47 elided
```

#### 2.4

```sql
scala> spark.sql("set;").show
+---+-----------+
|key|      value|
+---+-----------+
|  ;|<undefined>|
+---+-----------+

scala> spark.sql("set a;").show
+---+-----------+
|key|      value|
+---+-----------+
| a;|<undefined>|
+---+-----------+
```

In this PR,
1.  make `set spark.sql.abc=2; select 1;` in `SparkSession.sql` fail directly, user should call `.sql` for each statement separately.
2. make the semicolon as the separator of statements, and if users want to use it as part of the property value, shall use quotes too.

### Why are the changes needed?

1. disambiguation for  `SparkSession.sql`
2. make semicolon work same both w/ `SET` and other statements

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

yes,
the semicolon works as a separator of statements now, it will be trimmed if it is at the end of the statement and fail the statement if it is in the middle. you need to use quotes if you want it to be part of the property value

### How was this patch tested?

new tests

Closes #30332 from yaooqinn/SPARK-33419.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-13 06:58:16 +00:00
ulysses 82a21d2a3e [SPARK-33433][SQL] Change Aggregate max rows to 1 if grouping is empty
### What changes were proposed in this pull request?

Change `Aggregate` max rows to 1 if grouping is empty.

### Why are the changes needed?

If `Aggregate` grouping is empty, the result is always one row.

Then we don't need push down limit in `LimitPushDown` with such case
```
select count(*) from t1
union
select count(*) from t2
limit 1
```

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

No.

### How was this patch tested?

Add test.

Closes #30356 from ulysses-you/SPARK-33433.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-11-13 15:57:07 +09:00
Max Gekk 539c2deb89 [SPARK-33426][SQL][TESTS] Unify Hive SHOW TABLES tests
### What changes were proposed in this pull request?
1. Create the separate test suite `org.apache.spark.sql.hive.execution.command.ShowTablesSuite`.
2. Re-use V1 SHOW TABLES tests added by https://github.com/apache/spark/pull/30287 in the Hive test suites.
3. Add new test case for the pattern `'table_name_1*|table_name_2*'` in the common test suite.

### Why are the changes needed?
To test V1 + common  SHOW TABLES tests in Hive.

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

### How was this patch tested?
By running v1/v2 and Hive v1 `ShowTablesSuite`:
```
$  build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowTablesSuite"
```

Closes #30340 from MaxGekk/show-tables-hive-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-13 05:15:13 +00:00
Liang-Chi Hsieh 2c64b731ae
[SPARK-33259][SS] Disable streaming query with possible correctness issue by default
### What changes were proposed in this pull request?

This patch proposes to disable the streaming query with possible correctness issue in chained stateful operators. The behavior can be controlled by a SQL config, so if users understand the risk and still want to run the query, they can disable the check.

### Why are the changes needed?

The possible correctness in chained stateful operators in streaming query is not straightforward for users. From users perspective, it will be considered as a Spark bug. It is also possible the worse case, users are not aware of the correctness issue and use wrong results.

A better approach should be to disable such queries and let users choose to run the query if they understand there is such risk, instead of implicitly running the query and let users to find out correctness issue by themselves and report this known to Spark community.

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

Yes. Streaming query with possible correctness issue will be blocked to run, except for users explicitly disable the SQL config.

### How was this patch tested?

Unit test.

Closes #30210 from viirya/SPARK-33259.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-12 15:31:57 -08:00
Chao Sun cf3b6551ce
[SPARK-33435][SQL] DSv2: REFRESH TABLE should invalidate caches referencing the table
### What changes were proposed in this pull request?

This changes `RefreshTableExec` in DSv2 to also invalidate caches with references to the target table to be refreshed. The change itself is similar to what's done in #30211. Note that though, since we currently don't support caching a DSv2 table directly, this doesn't add recache logic as in the DSv1 impl. I marked it as a TODO for now.

### Why are the changes needed?

Currently the behavior in DSv1 and DSv2 is inconsistent w.r.t refreshing table: in DSv1 we invalidate both metadata cache as well as all table caches that are related to the table, but in DSv2 we only do the former. This addresses the issue and make the behavior consistent.

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

Yes, now refreshing a v2 table also invalidate all the related caches.

### How was this patch tested?

Added a new UT.

Closes #30359 from sunchao/SPARK-33435.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-12 15:22:56 -08:00
Linhong Liu 1baf0d5c9b [SPARK-33140][SQL][FOLLOW-UP] change val to def in object rule
### What changes were proposed in this pull request?
In #30097, many rules changed from case class to object, but if the rule
is stateful, there will be a problem. For example, if an object rule uses a
`val` to refer to a config, it will be unchanged after initialization even if
other spark session uses a different config value.

### Why are the changes needed?
Avoid potential bug

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

### How was this patch tested?
Existing UT

Closes #30354 from linhongliu-db/SPARK-33140-followup-2.

Lead-authored-by: Linhong Liu <67896261+linhongliu-db@users.noreply.github.com>
Co-authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-13 01:10:28 +09:00
gengjiaan 2f07c56810 [SPARK-33278][SQL] Improve the performance for FIRST_VALUE
### What changes were proposed in this pull request?
https://github.com/apache/spark/pull/29800 provides a performance improvement for `NTH_VALUE`.
`FIRST_VALUE` also could use the `UnboundedOffsetWindowFunctionFrame` and `UnboundedPrecedingOffsetWindowFunctionFrame`.

### Why are the changes needed?
Improve the performance for `FIRST_VALUE`.

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

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

Closes #30178 from beliefer/SPARK-33278.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-12 14:59:22 +00:00
ulysses a3d2954662 [SPARK-33421][SQL] Support Greatest and Least in Expression Canonicalize
### What changes were proposed in this pull request?

Add `Greatest` and `Least` check in `Canonicalize`.

### Why are the changes needed?

The children of both `Greatest` and `Least` are order Irrelevant.

Let's say we have `greatest(1, 2)` and `greatest(2, 1)`. We can get the same canonicalized expression in this case.

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

No.

### How was this patch tested?

Add test.

Closes #30330 from ulysses-you/SPARK-33421.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-12 20:26:33 +09:00
xuewei.linxuewei 6d31daeb6a [SPARK-33386][SQL] Accessing array elements in ElementAt/Elt/GetArrayItem should failed if index is out of bound
### What changes were proposed in this pull request?

Instead of returning NULL, throws runtime ArrayIndexOutOfBoundsException when ansiMode is enable for `element_at`,`elt`, `GetArrayItem` functions.

### Why are the changes needed?

For ansiMode.

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

When `spark.sql.ansi.enabled` = true, Spark will throw `ArrayIndexOutOfBoundsException` if out-of-range index when accessing array elements

### How was this patch tested?

Added UT and existing UT.

Closes #30297 from leanken/leanken-SPARK-33386.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-12 08:50:32 +00:00
Yuanjian Li 9f983a68f1 [SPARK-30294][SS][FOLLOW-UP] Directly override RDD methods
### Why are the changes needed?
Follow the comment: https://github.com/apache/spark/pull/26935#discussion_r514697997

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

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

Closes #30344 from xuanyuanking/SPARK-30294-follow.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-12 12:22:25 +09:00
Max Gekk 7e867298fe
[SPARK-33404][SQL][FOLLOWUP] Update benchmark results for date_trunc
### What changes were proposed in this pull request?
Updated results of `DateTimeBenchmark` in the environment:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge (spot instance) |
| AMI | ami-06f2f779464715dc5 (ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1) |
| Java | OpenJDK8/11 installed by`sudo add-apt-repository ppa:openjdk-r/ppa` & `sudo apt install openjdk-11-jdk`|

### Why are the changes needed?
The fix https://github.com/apache/spark/pull/30303 slowed down `date_trunc`. This PR updates benchmark results to have actual info about performance of `date_trunc`.

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

### How was this patch tested?
By regenerating benchmark results:
```
$ SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.DateTimeBenchmark"
```

Closes #30338 from MaxGekk/fix-trunc_date-benchmark.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-11 08:50:43 -08:00
stczwd 1eb236b936 [SPARK-32512][SQL] add alter table add/drop partition command for datasourcev2
### What changes were proposed in this pull request?
This patch is trying to add `AlterTableAddPartitionExec` and `AlterTableDropPartitionExec` with the new table partition API, defined in #28617.

### Does this PR introduce _any_ user-facing change?
Yes. User can use `alter table add partition` or `alter table drop partition` to create/drop partition in V2Table.

### How was this patch tested?
Run suites and fix old tests.

Closes #29339 from stczwd/SPARK-32512-new.

Lead-authored-by: stczwd <qcsd2011@163.com>
Co-authored-by: Jacky Lee <qcsd2011@163.com>
Co-authored-by: Jackey Lee <qcsd2011@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-11 09:30:42 +00:00
Wenchen Fan 8760032f4f [SPARK-33412][SQL] OverwriteByExpression should resolve its delete condition based on the table relation not the input query
### What changes were proposed in this pull request?

Make a special case in `ResolveReferences`, which resolves `OverwriteByExpression`'s condition expression based on the table relation instead of the input query.

### Why are the changes needed?

The condition expression is passed to the table implementation at the end, so we should resolve it using table schema. Previously it works because we have a hack in `ResolveReferences` to delay the resolution if `outputResolved == false`. However, this hack doesn't work for tables accepting any schema like https://github.com/delta-io/delta/pull/521 . We may wrongly resolve the delete condition using input query's outout columns which don't match the table column names.

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

No

### How was this patch tested?

existing tests and updated test in v2 write.

Closes #30318 from cloud-fan/v2-write.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-11 16:13:21 +09:00
Takeshi Yamamuro 4b367976a8 [SPARK-33417][SQL][TEST] Correct the behaviour of query filters in TPCDSQueryBenchmark
### What changes were proposed in this pull request?

This PR intends to fix the behaviour of query filters in `TPCDSQueryBenchmark`. We can use an option `--query-filter` for selecting TPCDS queries to run, e.g., `--query-filter q6,q8,q13`. But, the current master has a weird behaviour about the option. For example, if we pass `--query-filter q6` so as to run the TPCDS q6 only, `TPCDSQueryBenchmark` runs `q6` and `q6-v2.7` because the `filterQueries` method does not respect the name suffix. So, there is no way now to run the TPCDS q6 only.

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

Manually checked.

Closes #30324 from maropu/FilterBugInTPCDSQueryBenchmark.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-11-11 15:24:05 +09:00
Terry Kim 6d5d030957 [SPARK-33414][SQL] Migrate SHOW CREATE TABLE command to use UnresolvedTableOrView to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `SHOW CREATE TABLE` to use `UnresolvedTableOrView` to resolve the table identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).

Note that `SHOW CREATE TABLE` works only with a v1 table and a permanent view, and not supported for v2 tables.

### Why are the changes needed?

The changes allow consistent resolution behavior when resolving the table identifier. For example, the following is the current behavior:
```scala
sql("CREATE TEMPORARY VIEW t AS SELECT 1")
sql("CREATE DATABASE db")
sql("CREATE TABLE t (key INT, value STRING) USING hive")
sql("USE db")
sql("SHOW CREATE TABLE t AS SERDE") // Succeeds
```
With this change, `SHOW CREATE TABLE ... AS SERDE` above fails with the following:
```
org.apache.spark.sql.AnalysisException: t is a temp view not table or permanent view.; line 1 pos 0
  at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
  at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTempViews$$anonfun$apply$7.$anonfun$applyOrElse$43(Analyzer.scala:883)
  at scala.Option.map(Option.scala:230)
```
, which is expected since temporary view is resolved first and `SHOW CREATE TABLE ... AS SERDE` doesn't support a temporary view.

Note that there is no behavior change for `SHOW CREATE TABLE` without `AS SERDE` since it was already resolving to a temporary view first. See below for more detail.

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

After this PR, `SHOW CREATE TABLE t AS SERDE` is resolved to a temp view `t` instead of table `db.t` in the above scenario.

Note that there is no behavior change for `SHOW CREATE TABLE` without `AS SERDE`, but the exception message changes from `SHOW CREATE TABLE is not supported on a temporary view` to `t is a temp view not table or permanent view`.

### How was this patch tested?

Updated existing tests.

Closes #30321 from imback82/show_create_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-11 05:54:27 +00:00
Max Gekk 1e2eeda20e [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests
### What changes were proposed in this pull request?
In the PR, I propose to gather common `SHOW TABLES` tests into one trait `org.apache.spark.sql.execution.command.ShowTablesSuite`, and put datasource specific tests to the `v1.ShowTablesSuite` and `v2.ShowTablesSuite`. Also tests for parsing `SHOW TABLES` are extracted to `ShowTablesParserSuite`.

### Why are the changes needed?
- The unification will allow to run common `SHOW TABLES` tests for both DSv1 and DSv2
- We can detect missing features and differences between DSv1 and DSv2 implementations.

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

### How was this patch tested?
By running new test suites:
- `org.apache.spark.sql.execution.command.v1.ShowTablesSuite`
- `org.apache.spark.sql.execution.command.v2.ShowTablesSuite`
- `ShowTablesParserSuite`

Closes #30287 from MaxGekk/unify-dsv1_v2-tests.

Lead-authored-by: Max Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-11 05:26:46 +00:00
ulysses 5197c5d2e7 [SPARK-33390][SQL] Make Literal support char array
### What changes were proposed in this pull request?

Make Literal support char array.

### Why are the changes needed?

We always use `Literal()` to create foldable value, and `char[]` is a usual data type. We can make it easy that support create String Literal with `char[]`.

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

Yes, user can call `Literal()` with `char[]`.

### How was this patch tested?

Add test.

Closes #30295 from ulysses-you/SPARK-33390.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-11 11:39:11 +09:00
Utkarsh 46346943bb [SPARK-33404][SQL] Fix incorrect results in date_trunc expression
### What changes were proposed in this pull request?
The following query produces incorrect results:
```
SELECT date_trunc('minute', '1769-10-17 17:10:02')
```
Spark currently incorrectly returns
```
1769-10-17 17:10:02
```
against the expected return value of
```
1769-10-17 17:10:00
```
**Steps to repro**
Run the following commands in spark-shell:
```
spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
spark.sql("SELECT date_trunc('minute', '1769-10-17 17:10:02')").show()
```
This happens as `truncTimestamp` in package `org.apache.spark.sql.catalyst.util.DateTimeUtils` incorrectly assumes that time zone offsets can never have the granularity of a second and thus does not account for time zone adjustment when truncating the given timestamp to `minute`.
This assumption is currently used when truncating the timestamps to `microsecond, millisecond, second, or minute`.

This PR fixes this issue and always uses time zone knowledge when truncating timestamps regardless of the truncation unit.

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

### How was this patch tested?
Added new tests to `DateTimeUtilsSuite` which previously failed and pass now.

Closes #30303 from utkarsh39/trunc-timestamp-fix.

Authored-by: Utkarsh <utkarsh.agarwal@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-11-11 09:28:59 +09:00
Liang-Chi Hsieh 6fa80ed1dd [SPARK-33337][SQL] Support subexpression elimination in branches of conditional expressions
### What changes were proposed in this pull request?

Currently we skip subexpression elimination in branches of conditional expressions including `If`, `CaseWhen`, and `Coalesce`. Actually we can do subexpression elimination for such branches if the subexpression is common across all branches. This patch proposes to support subexpression elimination in branches of conditional expressions.

### Why are the changes needed?

We may miss subexpression elimination chances in branches of conditional expressions. This kind of subexpression is frequently seen. It may be written manually by users or come from query optimizer. For example, project collapsing could embed expressions between two `Project`s and produces conditional expression like:

```
CASE WHEN jsonToStruct(json).a = '1' THEN 1.0 WHEN jsonToStruct(json).a = '2' THEN 2.0 ... ELSE 1.2 END
```

If `jsonToStruct(json)` is time-expensive expression, we don't eliminate the duplication and waste time on running it repeatedly now.

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

No

### How was this patch tested?

Unit test.

Closes #30245 from viirya/SPARK-33337.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-11-10 16:17:00 -08:00
Chao Sun 3165ca742a [SPARK-33376][SQL] Remove the option of "sharesHadoopClasses" in Hive IsolatedClientLoader
### What changes were proposed in this pull request?

This removes the `sharesHadoopClasses` flag from `IsolatedClientLoader` in Hive module.

### Why are the changes needed?

Currently, when initializing `IsolatedClientLoader`, users can set the `sharesHadoopClasses` flag to decide whether the `HiveClient` created should share Hadoop classes with Spark itself or not. In the latter case, the client will only load Hadoop classes from the Hive dependencies.

There are two reasons to remove this:
1. this feature is currently used in two cases: 1) unit tests, 2) when the Hadoop version defined in Maven can not be found when `spark.sql.hive.metastore.jars` is equal to "maven", which could be very rare.
2. when `sharesHadoopClasses` is false, Spark doesn't really only use Hadoop classes from Hive jars: we also download `hadoop-client` jar and put all the sub-module jars (e.g., `hadoop-common`, `hadoop-hdfs`) together with the Hive jars, and the Hadoop version used by `hadoop-client` is the same version used by Spark itself. As result, we're mixing two versions of Hadoop jars in the classpath, which could potentially cause issues, especially considering that the default Hadoop version is already 3.2.0 while most Hive versions supported by the `IsolatedClientLoader` is still using Hadoop 2.x or even lower.

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

This affects Spark users in one scenario: when `spark.sql.hive.metastore.jars` is set to `maven` AND the Hadoop version specified in pom file cannot be downloaded, currently the behavior is to switch to _not_ share Hadoop classes, but with the PR it will share Hadoop classes with Spark.

### How was this patch tested?

Existing UTs.

Closes #30284 from sunchao/SPARK-33376.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-10 15:41:04 +00:00
angerszhu 34f5e7ce77 [SPARK-33302][SQL] Push down filters through Expand
### What changes were proposed in this pull request?
Push down filter through expand.  For case below:
```
create table t1(pid int, uid int, sid int, dt date, suid int) using parquet;
create table t2(pid int, vs int, uid int, csid int) using parquet;

SELECT
       years,
       appversion,
       SUM(uusers) AS users
FROM   (SELECT
               Date_trunc('year', dt)          AS years,
               CASE
                 WHEN h.pid = 3 THEN 'iOS'
                 WHEN h.pid = 4 THEN 'Android'
                 ELSE 'Other'
               END                             AS viewport,
               h.vs                            AS appversion,
               Count(DISTINCT u.uid)           AS uusers
               ,Count(DISTINCT u.suid)         AS srcusers
        FROM   t1 u
               join t2 h
                 ON h.uid = u.uid
        GROUP  BY 1,
                  2,
                  3) AS a
WHERE  viewport = 'iOS'
GROUP  BY 1,
          2
```

Plan. before this pr:
```
== Physical Plan ==
*(5) HashAggregate(keys=[years#30, appversion#32], functions=[sum(uusers#33L)])
+- Exchange hashpartitioning(years#30, appversion#32, 200), true, [id=#251]
   +- *(4) HashAggregate(keys=[years#30, appversion#32], functions=[partial_sum(uusers#33L)])
      +- *(4) HashAggregate(keys=[date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12], functions=[count(if ((gid#44 = 1)) u.`uid`#47 else null)])
         +- Exchange hashpartitioning(date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12, 200), true, [id=#246]
            +- *(3) HashAggregate(keys=[date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12], functions=[partial_count(if ((gid#44 = 1)) u.`uid`#47 else null)])
               +- *(3) HashAggregate(keys=[date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12, u.`uid`#47, u.`suid`#48, gid#44], functions=[])
                  +- Exchange hashpartitioning(date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12, u.`uid`#47, u.`suid`#48, gid#44, 200), true, [id=#241]
                     +- *(2) HashAggregate(keys=[date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12, u.`uid`#47, u.`suid`#48, gid#44], functions=[])
                        +- *(2) Filter (CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46 = iOS)
                           +- *(2) Expand [ArrayBuffer(date_trunc(year, cast(dt#9 as timestamp), Some(Etc/GMT+7)), CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END, vs#12, uid#7, null, 1), ArrayBuffer(date_trunc(year, cast(dt#9 as timestamp), Some(Etc/GMT+7)), CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END, vs#12, null, suid#10, 2)], [date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12, u.`uid`#47, u.`suid`#48, gid#44]
                              +- *(2) Project [uid#7, dt#9, suid#10, pid#11, vs#12]
                                 +- *(2) BroadcastHashJoin [uid#7], [uid#13], Inner, BuildRight
                                    :- *(2) Project [uid#7, dt#9, suid#10]
                                    :  +- *(2) Filter isnotnull(uid#7)
                                    :     +- *(2) ColumnarToRow
                                    :        +- FileScan parquet default.t1[uid#7,dt#9,suid#10] Batched: true, DataFilters: [isnotnull(uid#7)], Format: Parquet, Location: InMemoryFileIndex[file:/root/spark-3.0.0-bin-hadoop3.2/spark-warehouse/t1], PartitionFilters: [], PushedFilters: [IsNotNull(uid)], ReadSchema: struct<uid:int,dt:date,suid:int>
                                    +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint))), [id=#233]
                                       +- *(1) Project [pid#11, vs#12, uid#13]
                                          +- *(1) Filter isnotnull(uid#13)
                                             +- *(1) ColumnarToRow
                                                +- FileScan parquet default.t2[pid#11,vs#12,uid#13] Batched: true, DataFilters: [isnotnull(uid#13)], Format: Parquet, Location: InMemoryFileIndex[file:/root/spark-3.0.0-bin-hadoop3.2/spark-warehouse/t2], PartitionFilters: [], PushedFilters: [IsNotNull(uid)], ReadSchema: struct<pid:int,vs:int,uid:int>
```

Plan. after. this pr. :
```
== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
+- HashAggregate(keys=[years#0, appversion#2], functions=[sum(uusers#3L)], output=[years#0, appversion#2, users#5L])
   +- Exchange hashpartitioning(years#0, appversion#2, 5), true, [id=#71]
      +- HashAggregate(keys=[years#0, appversion#2], functions=[partial_sum(uusers#3L)], output=[years#0, appversion#2, sum#22L])
         +- HashAggregate(keys=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12], functions=[count(distinct uid#7)], output=[years#0, appversion#2, uusers#3L])
            +- Exchange hashpartitioning(date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, 5), true, [id=#67]
               +- HashAggregate(keys=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12], functions=[partial_count(distinct uid#7)], output=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, count#27L])
                  +- HashAggregate(keys=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, uid#7], functions=[], output=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, uid#7])
                     +- Exchange hashpartitioning(date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, uid#7, 5), true, [id=#63]
                        +- HashAggregate(keys=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles)) AS date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END AS CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, uid#7], functions=[], output=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, uid#7])
                           +- Project [uid#7, dt#9, pid#11, vs#12]
                              +- BroadcastHashJoin [uid#7], [uid#13], Inner, BuildRight, false
                                 :- Filter isnotnull(uid#7)
                                 :  +- FileScan parquet default.t1[uid#7,dt#9] Batched: true, DataFilters: [isnotnull(uid#7)], Format: Parquet, Location: InMemoryFileIndex[file:/private/var/folders/4l/7_c5c97s1_gb0d9_d6shygx00000gn/T/warehouse-c069d87..., PartitionFilters: [], PushedFilters: [IsNotNull(uid)], ReadSchema: struct<uid:int,dt:date>
                                 +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[2, int, false] as bigint)),false), [id=#58]
                                    +- Filter ((CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END = iOS) AND isnotnull(uid#13))
                                       +- FileScan parquet default.t2[pid#11,vs#12,uid#13] Batched: true, DataFilters: [(CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END = iOS), isnotnull..., Format: Parquet, Location: InMemoryFileIndex[file:/private/var/folders/4l/7_c5c97s1_gb0d9_d6shygx00000gn/T/warehouse-c069d87..., PartitionFilters: [], PushedFilters: [IsNotNull(uid)], ReadSchema: struct<pid:int,vs:int,uid:int>

```

### Why are the changes needed?
Improve  performance, filter more data.

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

### How was this patch tested?
Added UT

Closes #30278 from AngersZhuuuu/SPARK-33302.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-10 14:40:24 +00:00
Chao Sun 4934da56bc [SPARK-33305][SQL] DSv2: DROP TABLE command should also invalidate cache
### What changes were proposed in this pull request?

This changes `DropTableExec` to also invalidate caches referencing the table to be dropped, in a cascading manner.

### Why are the changes needed?

In DSv1, `DROP TABLE` command also invalidate caches as described in [SPARK-19765](https://issues.apache.org/jira/browse/SPARK-19765). However in DSv2 the same command only drops the table but doesn't handle the caches. This could lead to correctness issue.

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

Yes. Now DSv2 `DROP TABLE` command also invalidates cache.

### How was this patch tested?

Added a new UT

Closes #30211 from sunchao/SPARK-33305.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-10 14:37:42 +00:00
xuewei.linxuewei e3a768dd79 [SPARK-33391][SQL] element_at with CreateArray not respect one based index
### What changes were proposed in this pull request?

element_at with CreateArray not respect one based index.

repo step:

```
var df = spark.sql("select element_at(array(3, 2, 1), 0)")
df.printSchema()

df = spark.sql("select element_at(array(3, 2, 1), 1)")
df.printSchema()

df = spark.sql("select element_at(array(3, 2, 1), 2)")
df.printSchema()

df = spark.sql("select element_at(array(3, 2, 1), 3)")
df.printSchema()

root
– element_at(array(3, 2, 1), 0): integer (nullable = false)

root
– element_at(array(3, 2, 1), 1): integer (nullable = false)

root
– element_at(array(3, 2, 1), 2): integer (nullable = false)

root
– element_at(array(3, 2, 1), 3): integer (nullable = true)

correct answer should be
0 true which is outOfBounds return default true.
1 false
2 false
3 false

```

For expression eval, it respect the oneBasedIndex, but within checking the nullable, it calculates with zeroBasedIndex using `computeNullabilityFromArray`.

### Why are the changes needed?

Correctness issue.

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

No.

### How was this patch tested?

Added UT and existing UT.

Closes #30296 from leanken/leanken-SPARK-33391.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-10 07:23:47 +00:00
Yuanjian Li ad02ceda29 [SPARK-33244][SQL] Unify the code paths for spark.table and spark.read.table
### What changes were proposed in this pull request?

- Call `spark.read.table` in `spark.table`.
- Add comments for `spark.table` to emphasize it also support streaming temp view reading.

### Why are the changes needed?
The code paths of `spark.table` and `spark.read.table` should be the same. This behavior is broke in SPARK-32592 since we need to respect options in `spark.read.table` API.

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

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

Closes #30148 from xuanyuanking/SPARK-33244.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-10 05:46:45 +00:00
Terry Kim 90f6f39e42 [SPARK-33366][SQL] Migrate LOAD DATA command to use UnresolvedTable to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `LOAD DATA` to use `UnresolvedTable` to resolve the table identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).

Note that `LOAD DATA` is not supported for v2 tables.

### Why are the changes needed?

The changes allow consistent resolution behavior when resolving the table identifier. For example, the following is the current behavior:
```scala
sql("CREATE TEMPORARY VIEW t AS SELECT 1")
sql("CREATE DATABASE db")
sql("CREATE TABLE t (key INT, value STRING) USING hive")
sql("USE db")
sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE t") // Succeeds
```
With this change, `LOAD DATA` above fails with the following:
```
org.apache.spark.sql.AnalysisException: t is a temp view not table.; line 1 pos 0
    at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
    at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTempViews$$anonfun$apply$7.$anonfun$applyOrElse$39(Analyzer.scala:865)
    at scala.Option.foreach(Option.scala:407)
```
, which is expected since temporary view is resolved first and `LOAD DATA` doesn't support a temporary view.

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

After this PR, `LOAD DATA ... t` is resolved to a temp view `t` instead of table `db.t` in the above scenario.

### How was this patch tested?

Updated existing tests.

Closes #30270 from imback82/load_data_cmd.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-10 05:28:06 +00:00
Gengliang Wang a1f84d8714 [SPARK-33369][SQL] DSV2: Skip schema inference in write if table provider supports external metadata
### What changes were proposed in this pull request?

When TableProvider.supportsExternalMetadata() is true, Spark will use the input Dataframe's schema in `DataframeWriter.save()`/`DataStreamWriter.start()` and skip schema/partitioning inference.

### Why are the changes needed?

For all the v2 data sources which are not FileDataSourceV2, Spark always infers the table schema/partitioning on `DataframeWriter.save()`/`DataStreamWriter.start()`.
The inference of table schema/partitioning can be expensive. However, there is no such trait or flag for indicating a V2 source can use the input DataFrame's schema on `DataframeWriter.save()`/`DataStreamWriter.start()`. We can resolve the problem by adding a new expected behavior for the method `TableProvider.supportsExternalMetadata()`.

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

Yes, a new behavior for the data source v2 API `TableProvider.supportsExternalMetadata()` when it returns true.

### How was this patch tested?

Unit test

Closes #30273 from gengliangwang/supportsExternalMetadata.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-10 04:43:32 +00:00
Gabor Somogyi 4ac8133866 [SPARK-33223][SS][UI] Structured Streaming Web UI state information
### What changes were proposed in this pull request?
Structured Streaming UI is not containing state information. In this PR I've added it.

### Why are the changes needed?
Missing state information.

### Does this PR introduce _any_ user-facing change?
Additional UI elements appear.

### How was this patch tested?
Existing unit tests + manual test.
<img width="1044" alt="Screenshot 2020-10-30 at 15 14 21" src="https://user-images.githubusercontent.com/18561820/97715405-a1797000-1ac2-11eb-886a-e3e6efa3af3e.png">

Closes #30151 from gaborgsomogyi/SPARK-33223.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-10 11:22:35 +09:00
Peter Toth 84dc374611 [SPARK-33303][SQL] Deduplicate deterministic PythonUDF calls
### What changes were proposed in this pull request?
This PR modifies the `ExtractPythonUDFs` rule to deduplicate deterministic PythonUDF calls.

Before this PR the dataframe: `df.withColumn("c", batchedPythonUDF(col("a"))).withColumn("d", col("c"))` has the plan:
```
*(1) Project [value#1 AS a#4, pythonUDF1#15 AS c#7, pythonUDF1#15 AS d#10]
+- BatchEvalPython [dummyUDF(value#1), dummyUDF(value#1)], [pythonUDF0#14, pythonUDF1#15]
   +- LocalTableScan [value#1]
```
After this PR the deterministic PythonUDF calls are deduplicated:
```
*(1) Project [value#1 AS a#4, pythonUDF0#14 AS c#7, pythonUDF0#14 AS d#10]
+- BatchEvalPython [dummyUDF(value#1)], [pythonUDF0#14]
   +- LocalTableScan [value#1]
```

### Why are the changes needed?
To fix a performance issue.

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

### How was this patch tested?
New and existing UTs.

Closes #30203 from peter-toth/SPARK-33303-deduplicate-deterministic-udf-calls.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-09 19:27:36 +09:00
Linhong Liu 4e1c89400d [SPARK-33140][SQL][FOLLOW-UP] Use sparkSession in AQE context when applying rules
### What changes were proposed in this pull request?
After #30097, all rules are using `SparkSession.active` to get `SQLConf`
and `SparkSession`. But in AQE, when applying the rules for the initial plan,
we should use the spark session in AQE context.

### Why are the changes needed?
Fix potential problem caused by using the wrong spark session

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

### How was this patch tested?
Existing ut

Closes #30294 from linhongliu-db/SPARK-33140-followup.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-09 09:44:58 +00:00
Yuming Wang 7a5647a93a [SPARK-33385][SQL] Support bucket pruning for IsNaN
### What changes were proposed in this pull request?

This pr add support bucket pruning on `IsNaN` predicate.

### Why are the changes needed?

Improve query performance.

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

No.

### How was this patch tested?

Unit test.

Closes #30291 from wangyum/SPARK-33385.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-09 09:20:31 +00:00
Yuming Wang 69799c514f [SPARK-33372][SQL] Fix InSet bucket pruning
### What changes were proposed in this pull request?

This pr fix `InSet` bucket pruning because of it's values should not be `Literal`:
cbd3fdea62/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala (L253-L255)

### Why are the changes needed?

Fix bug.

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

No.

### How was this patch tested?

Unit test and manual test:

```scala
spark.sql("select id as a, id as b from range(10000)").write.bucketBy(100, "a").saveAsTable("t")
spark.sql("select * from t where a in (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)").show
```

Before this PR | After this PR
-- | --
![image](https://user-images.githubusercontent.com/5399861/98380788-fb120980-2083-11eb-8fae-4e21ad873e9b.png) | ![image](https://user-images.githubusercontent.com/5399861/98381095-5ba14680-2084-11eb-82ca-2d780c85305c.png)

Closes #30279 from wangyum/SPARK-33372.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-09 08:32:51 +00:00
Wenchen Fan 98730b7ee2 [SPARK-33087][SQL] DataFrameWriterV2 should delegate table resolution to the analyzer
### What changes were proposed in this pull request?

This PR makes `DataFrameWriterV2` to create query plans with `UnresolvedRelation` and leave the table resolution work to the analyzer.

### Why are the changes needed?

Table resolution work should be done by the analyzer. After this PR, the behavior is more consistent between different APIs (DataFrameWriter, DataFrameWriterV2 and SQL). See the next section for behavior changes.

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

Yes.
1. writes to a temp view of v2 relation: previously it fails with table not found exception, now it works if the v2 relation is writable. This is consistent with `DataFrameWriter` and SQL INSERT.
2. writes to other temp views: previously it fails with table not found exception, now it fails with a more explicit error message, saying that writing to a temp view of non-v2-relation is not allowed.
3. writes to a view: previously it fails with table not writable error, now it fails with a more explicit error message, saying that writing to a view is not allowed.
4. writes to a v1 table: previously it fails with table not writable error, now it fails with a more explicit error message, saying that writing to a v1 table is not allowed. (We can allow it later, by falling back to v1 command)

### How was this patch tested?

new tests

Closes #29970 from cloud-fan/refactor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-09 08:08:00 +00:00
Huaxin Gao bfb257f078 [SPARK-32405][SQL] Apply table options while creating tables in JDBC Table Catalog
### What changes were proposed in this pull request?
Currently in JDBCTableCatalog, we ignore the table options when creating table.
```
    // TODO (SPARK-32405): Apply table options while creating tables in JDBC Table Catalog
    if (!properties.isEmpty) {
      logWarning("Cannot create JDBC table with properties, these properties will be " +
        "ignored: " + properties.asScala.map { case (k, v) => s"$k=$v" }.mkString("[", ", ", "]"))
    }
```

### Why are the changes needed?
need to apply the table options when we create table

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

### How was this patch tested?
add new test

Closes #30154 from huaxingao/table_options.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-09 07:02:14 +00:00
Liang-Chi Hsieh c269b53f07 [SPARK-33384][SS] Delete temporary file when cancelling writing to final path even underlying stream throwing error
### What changes were proposed in this pull request?

In `RenameBasedFSDataOutputStream.cancel`, we do two things: closing underlying stream and delete temporary file, in a single try/catch block. Closing `OutputStream` could possibly throw `IOException` so we possibly missing deleting temporary file.

This patch proposes to delete temporary even underlying stream throwing error.

### Why are the changes needed?

To avoid leaving temporary files during canceling writing in `RenameBasedFSDataOutputStream`.

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

No

### How was this patch tested?

Unit test.

Closes #30290 from viirya/SPARK-33384.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-11-08 18:44:26 -08:00
yangjie01 02fd52cfbc [SPARK-33352][CORE][SQL][SS][MLLIB][AVRO][K8S] Fix procedure-like declaration compilation warnings in Scala 2.13
### What changes were proposed in this pull request?
There are two similar compilation warnings about procedure-like declaration in Scala 2.13:

```
[WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:70: procedure syntax is deprecated for constructors: add `=`, as in method definition
```
and

```
[WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala:211: procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `run`'s return type
```

this pr is the first part to resolve SPARK-33352:

- For constructors method definition add `=` to convert to function syntax

- For without `return type` methods definition add `: Unit =` to convert to function syntax

### Why are the changes needed?
Eliminate compilation warnings in Scala 2.13 and this change should be compatible with Scala 2.12

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #30255 from LuciferYang/SPARK-29392-FOLLOWUP.1.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-11-08 12:51:48 -06:00
Hannah Amundson 1090b1b00a [SPARK-32860][DOCS][SQL] Updating documentation about map support in Encoders
### What changes were proposed in this pull request?

Javadocs updated for the encoder to include maps as a collection type

### Why are the changes needed?

The javadocs were not updated with fix SPARK-16706

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

Yes, the javadocs are updated

### How was this patch tested?

sbt was run to ensure it meets scalastyle

Closes #30274 from hannahkamundson/SPARK-32860.

Lead-authored-by: Hannah Amundson <amundson.hannah@heb.com>
Co-authored-by: Hannah <48397717+hannahkamundson@users.noreply.github.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-08 20:29:24 +09:00
Stuart White 09fa7ecae1 [SPARK-33291][SQL] Improve DataFrame.show for nulls in arrays and structs
### What changes were proposed in this pull request?
The changes in [SPARK-32501 Inconsistent NULL conversions to strings](https://issues.apache.org/jira/browse/SPARK-32501) introduced some behavior that I'd like to clean up a bit.

Here's sample code to illustrate the behavior I'd like to clean up:

```scala
val rows = Seq[String](null)
  .toDF("value")
  .withColumn("struct1", struct('value as "value1"))
  .withColumn("struct2", struct('value as "value1", 'value as "value2"))
  .withColumn("array1", array('value))
  .withColumn("array2", array('value, 'value))

// Show the DataFrame using the "first" codepath.
rows.show(truncate=false)
+-----+-------+-------------+------+--------+
|value|struct1|struct2      |array1|array2  |
+-----+-------+-------------+------+--------+
|null |{ null}|{ null, null}|[]    |[, null]|
+-----+-------+-------------+------+--------+

// Write the DataFrame to disk, then read it back and show it to trigger the "codegen" code path:
rows.write.parquet("rows")
spark.read.parquet("rows").show(truncate=false)

+-----+-------+-------------+-------+-------------+
|value|struct1|struct2      |array1 |array2       |
+-----+-------+-------------+-------+-------------+
|null |{ null}|{ null, null}|[ null]|[ null, null]|
+-----+-------+-------------+-------+-------------+
```

Notice:

1. If the first element of a struct is null, it is printed with a leading space (e.g. "\{ null\}").  I think it's preferable to print it without the leading space (e.g. "\{null\}").  This is consistent with how non-null values are printed inside a struct.
2. If the first element of an array is null, it is not printed at all in the first code path, and the "codegen" code path prints it with a leading space.  I think both code paths should be consistent and print it without a leading space (e.g. "[null]").

The desired result of this PR is to product the following output via both code paths:

```
+-----+-------+------------+------+------------+
|value|struct1|struct2     |array1|array2      |
+-----+-------+------------+------+------------+
|null |{null} |{null, null}|[null]|[null, null]|
+-----+-------+------------+------+------------+
```

This contribution is my original work and I license the work to the project under the project’s open source license.

### Why are the changes needed?

To correct errors and inconsistencies in how DataFrame.show() displays nulls inside arrays and structs.

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

Yes.  This PR changes what is printed out by DataFrame.show().

### How was this patch tested?

I added new test cases in CastSuite.scala to cover the cases addressed by this PR.

Closes #30189 from stwhit/show_nulls.

Authored-by: Stuart White <stuart.white1@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-11-06 13:12:35 -08:00
Terry Kim 68c032c246 [SPARK-33364][SQL] Introduce the "purge" option in TableCatalog.dropTable for v2 catalog
### What changes were proposed in this pull request?

This PR proposes to introduce the `purge` option in `TableCatalog.dropTable` so that v2 catalogs can use the option if needed.

Related discussion: https://github.com/apache/spark/pull/30079#discussion_r510594110

### Why are the changes needed?

Spark DDL supports passing the purge option to `DROP TABLE` command. However, the option is not used (ignored) for v2 catalogs.

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

This PR introduces a new API in `TableCatalog`.

### How was this patch tested?

Added a test.

Closes #30267 from imback82/purge_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-11-05 22:00:45 -08:00
Prashant Sharma 733a468726 [SPARK-33130][SQL] Support ALTER TABLE in JDBC v2 Table Catalog: add, update type and nullability of columns (MsSqlServer dialect)
### What changes were proposed in this pull request?

Override the default SQL strings for:
ALTER TABLE RENAME COLUMN
ALTER TABLE UPDATE COLUMN NULLABILITY
in the following MsSQLServer JDBC dialect according to official documentation.
Write MsSqlServer integration tests for JDBC.

### Why are the changes needed?

To add the support for alter table when interacting with MSSql Server.

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

### How was this patch tested?

added tests

Closes #30038 from ScrapCodes/mssql-dialect.

Authored-by: Prashant Sharma <prashsh1@in.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-06 05:46:38 +00:00
Wenchen Fan d16311051d [SPARK-32934][SQL][FOLLOW-UP] Refine class naming and code comments
### What changes were proposed in this pull request?

1. Rename `OffsetWindowSpec` to `OffsetWindowFunction`, as it's the base class for all offset based window functions.
2. Refine and add more comments.
3. Remove `isRelative` as it's useless.

### Why are the changes needed?

code refinement

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

no

### How was this patch tested?

existing tests

Closes #30261 from cloud-fan/window.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-06 05:20:25 +00:00
Dongjoon Hyun 90f35c663e [MINOR][SQL] Fix incorrect JIRA ID comments in Analyzer
### What changes were proposed in this pull request?

This PR fixes incorrect JIRA ids in `Analyzer.scala` introduced by  SPARK-31670 (https://github.com/apache/spark/pull/28490)
```scala
- // SPARK-31607: Resolve Struct field in selectedGroupByExprs/groupByExprs and aggregations
+ // SPARK-31670: Resolve Struct field in selectedGroupByExprs/groupByExprs and aggregations
```

### Why are the changes needed?

Fix the wrong information.

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

No.

### How was this patch tested?

This is a comment change. Manually review.

Closes #30269 from dongjoon-hyun/SPARK-31670-MINOR.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-06 12:46:26 +09:00
Wenchen Fan cd4e3d3b0c [SPARK-33360][SQL] Simplify DS v2 write resolution
### What changes were proposed in this pull request?

Removing duplicated code in `ResolveOutputRelation`, by adding `V2WriteCommand.withNewQuery`

### Why are the changes needed?

code cleanup

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

No

### How was this patch tested?

existing tests

Closes #30264 from cloud-fan/ds-minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-11-05 15:44:04 -08:00
Wenchen Fan 26ea417b14 [SPARK-33362][SQL] skipSchemaResolution should still require query to be resolved
### What changes were proposed in this pull request?

Fix a small bug in `V2WriteCommand.resolved`. It should always require the `table` and `query` to be resolved.

### Why are the changes needed?

To prevent potential bugs that we skip resolve the input query.

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

no

### How was this patch tested?

a new test

Closes #30265 from cloud-fan/ds-minor-2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-11-05 09:23:41 -08:00
Jungtaek Lim (HeartSaVioR) 21413b7dd4 [SPARK-30294][SS] Explicitly defines read-only StateStore and optimize for HDFSBackedStateStore
### What changes were proposed in this pull request?

There's a concept of 'read-only' and 'read+write' state store in Spark which is defined "implicitly". Spark doesn't prevent write for 'read-only' state store; Spark just assumes read-only stateful operator will not modify the state store. Given it's not defined explicitly, the instance of state store has to be implemented as 'read+write' even it's being used as 'read-only', which sometimes brings confusion.

For example, abort() in HDFSBackedStateStore - d38f816748/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala (L143-L155)

The comment sounds as if statement works differently between 'read-only' and 'read+write', but that's not true as both state store has state initialized as UPDATING (no difference). So 'read-only' state also creates the temporary file, initializes output streams to write to temporary file, closes output streams, and finally deletes the temporary file. This unnecessary operations are being done per batch/partition.

This patch explicitly defines 'read-only' StateStore, and enables state store provider to create 'read-only' StateStore instance if requested. Relevant code paths are modified, as well as 'read-only' StateStore implementation for HDFSBackedStateStore is introduced. The new implementation gets rid of unnecessary operations explained above.

In point of backward-compatibility view, the only thing being changed in public API side is `StateStoreProvider`. The trait `StateStoreProvider` has to be changed to allow requesting 'read-only' StateStore; this patch adds default implementation which leverages 'read+write' StateStore but wrapping with 'write-protected' StateStore instance, so that custom providers don't need to change their code to reflect the change. But if the providers can optimize for read-only workload, they'll be happy to make a change.

Please note that this patch makes ReadOnlyStateStore extend StateStore and being referred as StateStore, as StateStore is being used in so many places and it's not easy to support both traits if we differentiate them. So unfortunately these write methods are still exposed for read-only state; it just throws UnsupportedOperationException.

### Why are the changes needed?

The new API opens the chance to optimize read-only state store instance compared with read+write state store instance. HDFSBackedStateStoreProvider is modified to provide read-only version of state store which doesn't deal with temporary file as well as state machine.

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

Clearly "no" for most end users, and also "no" for custom state store providers as it doesn't touch trait `StateStore` as well as provides default implementation for added method in trait `StateStoreProvider`.

### How was this patch tested?

Modified UT. Existing UTs ensure the change doesn't break anything.

Closes #26935 from HeartSaVioR/SPARK-30294.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-05 18:21:17 +09:00
HyukjinKwon d530ed0ea8 Revert "[SPARK-33277][PYSPARK][SQL] Use ContextAwareIterator to stop consuming after the task ends"
This reverts commit b8a440f098.
2020-11-05 16:15:17 +09:00
Dongjoon Hyun 42c0b175ce [SPARK-33338][SQL] GROUP BY using literal map should not fail
### What changes were proposed in this pull request?

This PR aims to fix `semanticEquals` works correctly on `GetMapValue` expressions having literal maps with `ArrayBasedMapData` and `GenericArrayData`.

### Why are the changes needed?

This is a regression from Apache Spark 1.6.x.
```scala
scala> sc.version
res1: String = 1.6.3

scala> sqlContext.sql("SELECT map('k1', 'v1')[k] FROM t GROUP BY map('k1', 'v1')[k]").show
+---+
|_c0|
+---+
| v1|
+---+
```

Apache Spark 2.x ~ 3.0.1 raise`RuntimeException` for the following queries.
```sql
CREATE TABLE t USING ORC AS SELECT map('k1', 'v1') m, 'k1' k
SELECT map('k1', 'v1')[k] FROM t GROUP BY 1
SELECT map('k1', 'v1')[k] FROM t GROUP BY map('k1', 'v1')[k]
SELECT map('k1', 'v1')[k] a FROM t GROUP BY a
```

**BEFORE**
```scala
Caused by: java.lang.RuntimeException: Couldn't find k#3 in [keys: [k1], values: [v1][k#3]#6]
	at scala.sys.package$.error(package.scala:27)
	at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:85)
	at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:79)
	at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
```

**AFTER**
```sql
spark-sql> SELECT map('k1', 'v1')[k] FROM t GROUP BY 1;
v1
Time taken: 1.278 seconds, Fetched 1 row(s)
spark-sql> SELECT map('k1', 'v1')[k] FROM t GROUP BY map('k1', 'v1')[k];
v1
Time taken: 0.313 seconds, Fetched 1 row(s)
spark-sql> SELECT map('k1', 'v1')[k] a FROM t GROUP BY a;
v1
Time taken: 0.265 seconds, Fetched 1 row(s)
```

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

No.

### How was this patch tested?

Pass the CIs with the newly added test case.

Closes #30246 from dongjoon-hyun/SPARK-33338.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-11-04 08:35:10 -08:00
Erik Krogen ff724d23b6 [SPARK-33214][TEST][HIVE] Stop HiveExternalCatalogVersionsSuite from using a hard-coded location to store localized Spark binaries
### What changes were proposed in this pull request?
This PR changes `HiveExternalCatalogVersionsSuite` to, by default, use a standard temporary directory to store the Spark binaries that it localizes. It additionally adds a new System property, `spark.test.cache-dir`, which can be used to define a static location into which the Spark binary will be localized to allow for sharing between test executions. If the System property is used, the downloaded binaries won't be deleted after the test runs.

### Why are the changes needed?
In SPARK-22356 (PR #19579), the `sparkTestingDir` used by `HiveExternalCatalogVersionsSuite` became hard-coded to enable re-use of the downloaded Spark tarball between test executions:
```
  // For local test, you can set `sparkTestingDir` to a static value like `/tmp/test-spark`, to
  // avoid downloading Spark of different versions in each run.
  private val sparkTestingDir = new File("/tmp/test-spark")
```
However this doesn't work, since it gets deleted every time:
```
  override def afterAll(): Unit = {
    try {
      Utils.deleteRecursively(wareHousePath)
      Utils.deleteRecursively(tmpDataDir)
      Utils.deleteRecursively(sparkTestingDir)
    } finally {
      super.afterAll()
    }
  }
```

It's bad that we're hard-coding to a `/tmp` directory, as in some cases this is not the proper place to store temporary files. We're not currently making any good use of it.

### Does this PR introduce _any_ user-facing change?
Developer-facing changes only, as this is in a test.

### How was this patch tested?
The test continues to execute as expected.

Closes #30122 from xkrogen/xkrogen-SPARK-33214-hiveexternalversioncatalogsuite-fix.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-04 06:51:54 +00:00
Terry Kim 0ad35ba5f8 [SPARK-33321][SQL] Migrate ANALYZE TABLE commands to use UnresolvedTableOrView to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `ANALYZE TABLE` and `ANALYZE TABLE ... FOR COLUMNS` to use `UnresolvedTableOrView` to resolve the table/view identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).

Note that `ANALYZE TABLE` is not supported for v2 tables.

### Why are the changes needed?

The changes allow consistent resolution behavior when resolving the table/view identifier. For example, the following is the current behavior:
```scala
sql("create temporary view t as select 1")
sql("create database db")
sql("create table db.t using csv as select 1")
sql("use db")
sql("ANALYZE TABLE t compute statistics") // Succeeds
```
With this change, ANALYZE TABLE above fails with the following:
```
    org.apache.spark.sql.AnalysisException: t is a temp view not table or permanent view.; line 1 pos 0
	at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTempViews$$anonfun$apply$7.$anonfun$applyOrElse$40(Analyzer.scala:872)
	at scala.Option.map(Option.scala:230)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTempViews$$anonfun$apply$7.applyOrElse(Analyzer.scala:870)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTempViews$$anonfun$apply$7.applyOrElse(Analyzer.scala:856)
```
, which is expected since temporary view is resolved first and ANALYZE TABLE doesn't support a temporary view.

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

After this PR, `ANALYZE TABLE t` is resolved to a temp view `t` instead of table `db.t`.

### How was this patch tested?

Updated existing tests.

Closes #30229 from imback82/parse_v1table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-04 06:50:37 +00:00
ulysses 1740b29b3f [SPARK-33323][SQL] Add query resolved check before convert hive relation
### What changes were proposed in this pull request?

Add query.resolved before  convert hive relation.

### Why are the changes needed?

For better error msg.
```
CREATE TABLE t STORED AS PARQUET AS
SELECT * FROM (
 SELECT c3 FROM (
  SELECT c1, c2 from values(1,2) t(c1, c2)
  )
)
```
 Before this PR, we get such error msg
```
org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to toAttribute on unresolved object, tree: *
  at org.apache.spark.sql.catalyst.analysis.Star.toAttribute(unresolved.scala:244)
  at org.apache.spark.sql.catalyst.plans.logical.Project$$anonfun$output$1.apply(basicLogicalOperators.scala:52)
  at org.apache.spark.sql.catalyst.plans.logical.Project$$anonfun$output$1.apply(basicLogicalOperators.scala:52)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.immutable.List.foreach(List.scala:392)
```

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

Yes, error msg changed.

### How was this patch tested?

Add test.

Closes #30230 from ulysses-you/SPARK-33323.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-04 05:01:39 +00:00
Wenchen Fan 034070a23a Revert "[SPARK-33248][SQL] Add a configuration to control the legacy behavior of whether need to pad null value when value size less then schema size"
This reverts commit 0c943cd2fb.
2020-11-04 12:30:38 +08:00
Chao Sun d900c6ff49 [SPARK-33293][SQL][FOLLOW-UP] Rename TableWriteExec to TableWriteExecHelper
### What changes were proposed in this pull request?

Rename `TableWriteExec` in `WriteToDataSourceV2Exec.scala` to `TableWriteExecHelper`.

### Why are the changes needed?

See [discussion](https://github.com/apache/spark/pull/30193#discussion_r516412653). The former is too general.

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

No

### How was this patch tested?

N/A

Closes #30235 from sunchao/SPARK-33293-2.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-11-03 14:53:01 -08:00
Max Gekk bdabf60fb4 [SPARK-33299][SQL][DOCS] Don't mention schemas in JSON format in docs for from_json
### What changes were proposed in this pull request?
Remove the JSON formatted schema from comments for `from_json()` in Scala/Python APIs.

Closes #30201

### Why are the changes needed?
Schemas in JSON format is internal (not documented). It shouldn't be recommenced for usage.

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

### How was this patch tested?
By linters.

Closes #30226 from MaxGekk/from_json-common-schema-parsing-2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-11-02 10:10:24 -08:00
Max Gekk eecebd0302 [SPARK-33306][SQL][FOLLOWUP] Group DateType and TimestampType together in needsTimeZone()
### What changes were proposed in this pull request?
In the PR, I propose to group `DateType` and `TimestampType` together in checking time zone needs in the `Cast.needsTimeZone()` method.

### Why are the changes needed?
To improve code maintainability.

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

### How was this patch tested?
By the existing test `"SPARK-33306: Timezone is needed when cast Date to String"`.

Closes #30223 from MaxGekk/WangGuangxin-SPARK-33306-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-11-02 10:07:18 -08:00
Yuming Wang 789d19cab5 [SPARK-33319][SQL][TEST] Add all built-in SerDes to HiveSerDeReadWriteSuite
### What changes were proposed in this pull request?

This pr add all built-in SerDes to `HiveSerDeReadWriteSuite`.

https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-RowFormats&SerDe

### Why are the changes needed?

We will upgrade Parquet, ORC and Avro, need to ensure compatibility.

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

No.

### How was this patch tested?

N/A

Closes #30228 from wangyum/SPARK-33319.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-11-02 08:34:50 -08:00
Cheng Su e52b858ef7 [SPARK-33027][SQL] Add DisableUnnecessaryBucketedScan rule to AQE
### What changes were proposed in this pull request?

As a followup comment from https://github.com/apache/spark/pull/29804#issuecomment-700650620 , here we add add the physical plan rule DisableUnnecessaryBucketedScan into AQE AdaptiveSparkPlanExec.queryStagePreparationRules, to make auto bucketed scan work with AQE.

The change is mostly in:
* `AdaptiveSparkPlanExec.scala`: add physical plan rule `DisableUnnecessaryBucketedScan`
* `DisableUnnecessaryBucketedScan.scala`: propagate logical plan link for the file source scan exec operator, otherwise we lose the logical plan link information when AQE is enabled, and will get exception [here](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala#L176). (for example, for query `SELECT * FROM bucketed_table` with AQE is enabled)
* `DisableUnnecessaryBucketedScanSuite.scala`: add new test suite for AQE enabled - `DisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE`, and changed some of tests to use `AdaptiveSparkPlanHelper.find/collect`, to make the plan verification work when AQE enabled.

### Why are the changes needed?

It's reasonable to add the support to allow disabling unnecessary bucketed scan with AQE is enabled, this helps optimize the query when AQE is enabled.

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

No.

### How was this patch tested?

Added unit test in `DisableUnnecessaryBucketedScanSuite`.

Closes #30200 from c21/auto-bucket-aqe.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-02 06:44:07 +00:00
Prashant Sharma 6226ccc092 [SPARK-33095] Follow up, support alter table column rename
### What changes were proposed in this pull request?

Support rename column for mysql dialect.

### Why are the changes needed?

At the moment, it does not work for mysql version 5.x. So, we should throw proper exception for that case.

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

Yes, `column rename` with mysql dialect should work correctly.

### How was this patch tested?

Added tests for rename column.
Ran the tests to pass with both versions of mysql.

* `export MYSQL_DOCKER_IMAGE_NAME=mysql:5.7.31`

* `export MYSQL_DOCKER_IMAGE_NAME=mysql:8.0`

Closes #30142 from ScrapCodes/mysql-dialect-rename.

Authored-by: Prashant Sharma <prashsh1@in.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-02 05:03:41 +00:00
Takuya UESHIN b8a440f098 [SPARK-33277][PYSPARK][SQL] Use ContextAwareIterator to stop consuming after the task ends
### What changes were proposed in this pull request?

As the Python evaluation consumes the parent iterator in a separate thread, it could consume more data from the parent even after the task ends and the parent is closed. Thus, we should use `ContextAwareIterator` to stop consuming after the task ends.

### Why are the changes needed?

Python/Pandas UDF right after off-heap vectorized reader could cause executor crash.

E.g.,:

```py
spark.range(0, 100000, 1, 1).write.parquet(path)

spark.conf.set("spark.sql.columnVector.offheap.enabled", True)

def f(x):
    return 0

fUdf = udf(f, LongType())

spark.read.parquet(path).select(fUdf('id')).head()
```

This is because, the Python evaluation consumes the parent iterator in a separate thread and it consumes more data from the parent even after the task ends and the parent is closed. If an off-heap column vector exists in the parent iterator, it could cause segmentation fault which crashes the executor.

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

No.

### How was this patch tested?

Added tests, and manually.

Closes #30177 from ueshin/issues/SPARK-33277/python_pandas_udf.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-01 20:28:12 +09:00
wangguangxin.cn 69c27f49ac [SPARK-33306][SQL] Timezone is needed when cast date to string
### What changes were proposed in this pull request?
When `spark.sql.legacy.typeCoercion.datetimeToString.enabled` is enabled, spark will cast date to string when compare date with string. In Spark3, timezone is needed when casting date to string as 72ad9dcd5d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala (L309).

Howerver, the timezone may not be set because `CastBase.needsTimeZone` returns false for this kind of casting.

A simple way to reproduce this is
```
spark-shell --conf spark.sql.legacy.typeCoercion.datetimeToString.enabled=true

```
when we execute the following sql,
```
select a.d1 from
(select to_date(concat('2000-01-0', id)) as d1 from range(1, 2)) a
join
(select concat('2000-01-0', id) as d2 from range(1, 2)) b
on a.d1 = b.d2
```
it will throw
```
java.util.NoSuchElementException: None.get
  at scala.None$.get(Option.scala:529)
  at scala.None$.get(Option.scala:527)
  at org.apache.spark.sql.catalyst.expressions.TimeZoneAwareExpression.zoneId(datetimeExpressions.scala:56)
  at org.apache.spark.sql.catalyst.expressions.TimeZoneAwareExpression.zoneId$(datetimeExpressions.scala:56)
  at org.apache.spark.sql.catalyst.expressions.CastBase.zoneId$lzycompute(Cast.scala:253)
  at org.apache.spark.sql.catalyst.expressions.CastBase.zoneId(Cast.scala:253)
  at org.apache.spark.sql.catalyst.expressions.CastBase.dateFormatter$lzycompute(Cast.scala:287)
  at org.apache.spark.sql.catalyst.expressions.CastBase.dateFormatter(Cast.scala:287)
```

### Why are the changes needed?
As described above, it's a bug here.

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

### How was this patch tested?
Add more UT

Closes #30213 from WangGuangxin/SPARK-33306.

Authored-by: wangguangxin.cn <wangguangxin.cn@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-31 15:14:46 -07:00