Commit graph

7369 commits

Author SHA1 Message Date
Wenchen Fan 0706e64c49 [SPARK-30098][SQL] Add a configuration to use default datasource as provider for CREATE TABLE command
### What changes were proposed in this pull request?

For CRETE TABLE [AS SELECT] command, creates native Parquet table if neither USING nor STORE AS is specified and `spark.sql.legacy.createHiveTableByDefault` is false.

This is a retry after we unify the CREATE TABLE syntax. It partially reverts d2bec5e265

This PR allows `CREATE EXTERNAL TABLE` when `LOCATION` is present. This was not allowed for data source tables before, which is an unnecessary behavior different with hive tables.

### Why are the changes needed?

Changing from Hive text table to native Parquet table has many benefits:
1. be consistent with `DataFrameWriter.saveAsTable`.
2. better performance
3. better support for nested types (Hive text table doesn't work well with nested types, e.g. `insert into t values struct(null)` actually inserts a null value not `struct(null)` if `t` is a Hive text table, which leads to wrong result)
4. better interoperability as Parquet is a more popular open file format.

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

No by default. If the config is set, the behavior change is described below:

Behavior-wise, the change is very small as the native Parquet table is also Hive-compatible. All the Spark DDL commands that works for hive tables also works for native Parquet tables, with two exceptions: `ALTER TABLE SET [SERDE | SERDEPROPERTIES]` and `LOAD DATA`.

char/varchar behavior has been taken care by https://github.com/apache/spark/pull/30412, and there is no behavior difference between data source and hive tables.

One potential issue is `CREATE TABLE ... LOCATION ...` while users want to directly access the files later. It's more like a corner case and the legacy config should be good enough.

Another potential issue is users may use Spark to create the table and then use Hive to add partitions with different serde. This is not allowed for Spark native tables.

### How was this patch tested?

Re-enable the tests

Closes #30554 from cloud-fan/create-table.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-03 15:24:44 +00:00
Gengliang Wang ff13f574e6 [SPARK-20044][SQL] Add new function DATE_FROM_UNIX_DATE and UNIX_DATE
### What changes were proposed in this pull request?

Add new functions DATE_FROM_UNIX_DATE and UNIX_DATE for conversion between Date type and Numeric types.

### Why are the changes needed?

1. Explicit conversion between Date type and Numeric types is disallowed in ANSI mode. We need to provide new functions for users to complete the conversion.

2. We have introduced new functions from Bigquery for conversion between Timestamp type and Numeric types: TIMESTAMP_SECONDS, TIMESTAMP_MILLIS, TIMESTAMP_MICROS , UNIX_SECONDS, UNIX_MILLIS, and UNIX_MICROS. It makes sense to add functions for conversion between Date type and Numeric types as well.

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

Yes, two new datetime functions are added.

### How was this patch tested?

Unit tests

Closes #30588 from gengliangwang/dateToNumber.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-03 14:04:08 +00:00
Yuanjian Li 878cc0e6e9
[SPARK-32896][SS][FOLLOW-UP] Rename the API to toTable
### What changes were proposed in this pull request?
As the discussion in https://github.com/apache/spark/pull/30521#discussion_r531463427, rename the API to `toTable`.

### Why are the changes needed?
Rename the API for further extension and accuracy.

### Does this PR introduce _any_ user-facing change?
Yes, it's an API change but the new API is not released yet.

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

Closes #30571 from xuanyuanking/SPARK-32896-follow.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-12-02 17:36:25 -08:00
uncleGen 4f96670358
[SPARK-31953][SS] Add Spark Structured Streaming History Server Support
### What changes were proposed in this pull request?

Add Spark Structured Streaming History Server Support.

### Why are the changes needed?

Add a streaming query history server plugin.

![image](https://user-images.githubusercontent.com/7402327/84248291-d26cfe80-ab3b-11ea-86d2-98205fa2bcc4.png)
![image](https://user-images.githubusercontent.com/7402327/84248347-e44ea180-ab3b-11ea-81de-eefe207656f2.png)
![image](https://user-images.githubusercontent.com/7402327/84248396-f0d2fa00-ab3b-11ea-9b0d-e410115471b0.png)

- Follow-ups
  - Query duration should not update in history UI.

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

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

Closes #28781 from uncleGen/SPARK-31953.

Lead-authored-by: uncleGen <hustyugm@gmail.com>
Co-authored-by: Genmao Yu <hustyugm@gmail.com>
Co-authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-12-02 17:11:51 -08:00
Gengliang Wang b76c6b759c
[SPARK-33627][SQL] Add new function UNIX_SECONDS, UNIX_MILLIS and UNIX_MICROS
### What changes were proposed in this pull request?

As https://github.com/apache/spark/pull/28534 adds functions from [BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/timestamp_functions) for converting numbers to timestamp, this PR is to add functions UNIX_SECONDS, UNIX_MILLIS and UNIX_MICROS for converting timestamp to numbers.

### Why are the changes needed?

1. Symmetry of the conversion functions
2. Casting timestamp type to numeric types is disallowed in ANSI mode, we should provide functions for users to complete the conversion.

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

3 new functions UNIX_SECONDS, UNIX_MILLIS and UNIX_MICROS for converting timestamp to long type.

### How was this patch tested?

Unit tests.

Closes #30566 from gengliangwang/timestampLong.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-02 12:44:39 -08:00
yi.wu a082f4600b [SPARK-33071][SPARK-33536][SQL] Avoid changing dataset_id of LogicalPlan in join() to not break DetectAmbiguousSelfJoin
### What changes were proposed in this pull request?

Currently, `join()` uses `withPlan(logicalPlan)` for convenient to call some Dataset functions. But it leads to the `dataset_id` inconsistent between the `logicalPlan` and the original `Dataset`(because `withPlan(logicalPlan)` will create a new Dataset with the new id and reset the `dataset_id` with the new id of the `logicalPlan`). As a result, it breaks the rule `DetectAmbiguousSelfJoin`.

In this PR, we propose to drop the usage of `withPlan` but use the `logicalPlan` directly so its `dataset_id` doesn't change.

Besides, this PR also removes related metadata (`DATASET_ID_KEY`,  `COL_POS_KEY`) when an `Alias` tries to construct its own metadata. Because the `Alias` is no longer a reference column after converting to an `Attribute`.  To achieve that, we add a new field, `deniedMetadataKeys`, to indicate the metadata that needs to be removed.

### Why are the changes needed?

For the query below, it returns the wrong result while it should throws ambiguous self join exception instead:

```scala
val emp1 = Seq[TestData](
  TestData(1, "sales"),
  TestData(2, "personnel"),
  TestData(3, "develop"),
  TestData(4, "IT")).toDS()
val emp2 = Seq[TestData](
  TestData(1, "sales"),
  TestData(2, "personnel"),
  TestData(3, "develop")).toDS()
val emp3 = emp1.join(emp2, emp1("key") === emp2("key")).select(emp1("*"))
emp1.join(emp3, emp1.col("key") === emp3.col("key"), "left_outer")
  .select(emp1.col("*"), emp3.col("key").as("e2")).show()

// wrong result
+---+---------+---+
|key|    value| e2|
+---+---------+---+
|  1|    sales|  1|
|  2|personnel|  2|
|  3|  develop|  3|
|  4|       IT|  4|
+---+---------+---+
```
This PR fixes the wrong behaviour.

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

Yes, users hit the exception instead of the wrong result after this PR.

### How was this patch tested?

Added a new unit test.

Closes #30488 from Ngone51/fix-self-join.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-02 17:51:22 +00:00
Cheng Su a4788ee8c6 [MINOR][SS] Rename auxiliary protected methods in StreamingJoinSuite
### What changes were proposed in this pull request?

Per request from https://github.com/apache/spark/pull/30395#issuecomment-735028698, here we remove `Windowed` from methods names `setupWindowedJoinWithRangeCondition` and `setupWindowedSelfJoin` as they don't join on time window.

### Why are the changes needed?

There's no such official name for `windowed join`, so this is to help avoid confusion for future developers.

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

No.

### How was this patch tested?

Existing unit tests.

Closes #30563 from c21/stream-minor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-12-02 15:28:16 +09:00
Cheng Su 51ebcd95a5 [SPARK-32863][SS] Full outer stream-stream join
### What changes were proposed in this pull request?

This PR is to add full outer stream-stream join, and the implementation of full outer join is:
* For left side input row, check if there's a match on right side state store.
  * if there's a match, output the joined row, o.w. output nothing. Put the row in left side state store.
* For right side input row, check if there's a match on left side state store.
  * if there's a match, output the joined row, o.w. output nothing. Put the row in right side state store.
* State store eviction: evict rows from left/right side state store below watermark, and output rows never matched before (a combination of left outer and right outer join).

### Why are the changes needed?

Enable more use cases for spark stream-stream join.

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

No.

### How was this patch tested?

Added unit tests in `UnsupportedOperationChecker.scala` and `StreamingJoinSuite.scala`.

Closes #30395 from c21/stream-foj.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-12-02 10:17:00 +09:00
Anton Okolnychyi c24f2b2d6a
[SPARK-33612][SQL] Add dataSourceRewriteRules batch to Optimizer
### What changes were proposed in this pull request?

This PR adds a new batch to the optimizer for executing rules that rewrite plans for data sources.

### Why are the changes needed?

Right now, we have a special place in the optimizer where we construct v2 scans. As time shows, we need more rewrite rules that would be executed after the operator optimization and before any stats-related rules for v2 tables. Not all rules will be specific to reads. One option is to rename the current batch into something more generic but it would require changing quite some places. That's why it seems better to introduce a new batch and use it for all rewrites. The name is generic so that we don't limit ourselves to v2 data sources only.

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

No.

### How was this patch tested?

The change is trivial and SPARK-23889 will depend on it.

Closes #30558 from aokolnychyi/spark-33612.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-01 09:27:46 -08:00
Prakhar Jain cf4ad212b1 [SPARK-33503][SQL] Refactor SortOrder class to allow multiple childrens
### What changes were proposed in this pull request?
This is a followup of #30302 . As part of this PR, sameOrderExpressions set is made part of children of SortOrder node - so that they don't need any special handling as done in #30302 .

### Why are the changes needed?
sameOrderExpressions should get same treatment as child. So making them part of children helps in transforming them easily.

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

### How was this patch tested?
Existing UTs

Closes #30430 from prakharjain09/SPARK-33400-sortorder-refactor.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-12-01 21:13:27 +09:00
gengjiaan 9273d4250d [SPARK-33045][SQL][FOLLOWUP] Support built-in function like_any and fix StackOverflowError issue
### What changes were proposed in this pull request?
Spark already support `LIKE ANY` syntax, but it will throw `StackOverflowError` if there are many elements(more than 14378 elements). We should implement built-in function for LIKE ANY 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 ANY` and avoid this issue.

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

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

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

Closes #30465 from beliefer/SPARK-33045-like_any-bak.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-01 11:48:30 +00:00
Huaxin Gao d38883c1d8 [SPARK-32405][SQL][FOLLOWUP] Throw Exception if provider is specified in JDBCTableCatalog create table
### What changes were proposed in this pull request?
Throw Exception if JDBC Table Catalog has provider in create table.

### Why are the changes needed?
JDBC Table Catalog doesn't support provider and we should throw Exception. Previously CREATE TABLE syntax forces people to specify a provider so we have to add a `USING_`. Now the problem was fix and we will throw Exception for provider.

### Does this PR introduce _any_ user-facing change?
Yes. We throw Exception if a provider is specified in CREATE TABLE for JDBC Table catalog.

### How was this patch tested?
Existing tests (remove `USING _`)

Closes #30544 from huaxingao/followup.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-01 11:38:42 +00:00
zky.zhoukeyong 1034815519 [SPARK-33572][SQL] Datetime building should fail if the year, month, ..., second combination is invalid
### What changes were proposed in this pull request?
Datetime building should fail if the year, month, ..., second combination is invalid, when ANSI mode is enabled. This patch should update MakeDate, MakeTimestamp and MakeInterval.

### 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 #30516 from waitinfuture/SPARK-33498.

Lead-authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Co-authored-by: waitinfuture <waitinfuture@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-01 11:07:16 +00:00
Jungtaek Lim (HeartSaVioR) 52e5cc46bc [SPARK-27188][SS] FileStreamSink: provide a new option to have retention on output files
### What changes were proposed in this pull request?

This patch proposes to provide a new option to specify time-to-live (TTL) for output file entries in FileStreamSink. TTL is defined via current timestamp - the last modified time for the file.

This patch will filter out outdated output files in metadata while compacting batches (other batches don't have functionality to clean entries), which helps metadata to not grow linearly, as well as filtered out files will be "eventually" no longer seen in reader queries which leverage File(Stream)Source.

### Why are the changes needed?

The metadata log greatly helps to easily achieve exactly-once but given the output path is open to arbitrary readers, there's no way to compact the metadata log, which ends up growing the metadata file as query runs for long time, especially for compacted batch.

Lots of end users have been reporting the issue: see comments in [SPARK-24295](https://issues.apache.org/jira/browse/SPARK-24295) and [SPARK-29995](https://issues.apache.org/jira/browse/SPARK-29995), and [SPARK-30462](https://issues.apache.org/jira/browse/SPARK-30462).
(There're some reports from end users which include their workarounds: SPARK-24295)

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

No, as the configuration is new and by default it is not applied.

### How was this patch tested?

New UT.

Closes #28363 from HeartSaVioR/SPARK-27188-v2.

Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-12-01 14:42:48 +09:00
Jungtaek Lim (HeartSaVioR) 2af2da5a4b [SPARK-30900][SS] FileStreamSource: Avoid reading compact metadata log twice if the query restarts from compact batch
### What changes were proposed in this pull request?

This patch addresses the case where compact metadata file is read twice in FileStreamSource during restarting query.

When restarting the query, there is a case which the query starts from compaction batch, and the batch has source metadata file to read. One case is that the previous query succeeded to read from inputs, but not finalized the batch for various reasons.

The patch finds the latest compaction batch when restoring from metadata log, and put entries for the batch into the file entry cache which would avoid reading compact batch file twice.

FileStreamSourceLog doesn't know about offset / commit metadata in checkpoint so doesn't know which exactly batch to start from, but in practice, only couple of latest batches are candidates to
be started from when restarting query. This patch leverages the fact to skip calculation if possible.

### Why are the changes needed?

Spark incurs unnecessary cost on reading the compact metadata file twice on some case, which may not be ignorable when the query has been processed huge number of files so far.

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

No.

### How was this patch tested?

New UT.

Closes #27649 from HeartSaVioR/SPARK-30900.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-12-01 13:11:14 +09:00
Kousuke Saruta c50fcac00e [SPARK-33607][SS][WEBUI] Input Rate timeline/histogram aren't rendered if built with Scala 2.13
### What changes were proposed in this pull request?

This PR fixes an issue that the histogram and timeline aren't rendered in the `Streaming Query Statistics` page if we built Spark with Scala 2.13.

![before-fix-the-issue](https://user-images.githubusercontent.com/4736016/100612855-f543d700-3356-11eb-90d9-ede57b8b3f4f.png)
![NaN_Error](https://user-images.githubusercontent.com/4736016/100612879-00970280-3357-11eb-97cf-43978bbe2d3a.png)

The reason is [`maxRecordRate` can be `NaN`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala#L371) for Scala 2.13.

The `NaN` is the result of [`query.recentProgress.map(_.inputRowsPerSecond).max`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala#L372) when the first element of `query.recentProgress.map(_.inputRowsPerSecond)` is `NaN`.
Actually, the comparison logic for `Double` type was changed in Scala 2.13.
https://github.com/scala/bug/issues/12107
https://github.com/scala/scala/pull/6410

So this issue happens as of Scala 2.13.

The root cause of the `NaN` is [here](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala#L164).
This `NaN` seems to be an initial value of `inputTimeSec` so I think `Double.PositiveInfinity` is suitable rather than `NaN` and this change can resolve this issue.

### Why are the changes needed?

To make sure we can use the histogram/timeline with Scala 2.13.

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

No.

### How was this patch tested?

First, I built with the following commands.
```
$ /dev/change-scala-version.sh 2.13
$ build/sbt -Phive -Phive-thriftserver -Pscala-2.13 package
```

Then, ran the following query (this is brought from #30427 ).
```
import org.apache.spark.sql.streaming.Trigger
val query = spark
  .readStream
  .format("rate")
  .option("rowsPerSecond", 1000)
  .option("rampUpTime", "10s")
  .load()
  .selectExpr("*", "CAST(CAST(timestamp AS BIGINT) - CAST((RAND() * 100000) AS BIGINT) AS TIMESTAMP) AS tsMod")
  .selectExpr("tsMod", "mod(value, 100) as mod", "value")
  .withWatermark("tsMod", "10 seconds")
  .groupBy(window($"tsMod", "1 minute", "10 seconds"), $"mod")
  .agg(max("value").as("max_value"), min("value").as("min_value"), avg("value").as("avg_value"))
  .writeStream
  .format("console")
  .trigger(Trigger.ProcessingTime("5 seconds"))
  .outputMode("append")
  .start()
```

Finally, I confirmed that the timeline and histogram are rendered.
![after-fix-the-issue](https://user-images.githubusercontent.com/4736016/100612736-c9285600-3356-11eb-856d-7e53cc656c36.png)

```

Closes #30546 from sarutak/ss-nan.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-12-01 11:45:32 +09:00
Max Gekk 030b3139da [SPARK-33569][SPARK-33452][SQL][FOLLOWUP] Fix a build error in ShowPartitionsExec
### What changes were proposed in this pull request?
Use `listPartitionIdentifiers ` instead of `listPartitionByNames` in `ShowPartitionsExec`. The `listPartitionByNames` was renamed by https://github.com/apache/spark/pull/30514.

### Why are the changes needed?
To fix build error.

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

### How was this patch tested?
By running tests for the `SHOW PARTITIONS` command:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowPartitionsSuite"
```

Closes #30553 from MaxGekk/fix-build-show-partitions-exec.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 16:40:36 +00:00
Max Gekk 6fd148fea8 [SPARK-33569][SQL] Remove getting partitions by an identifier prefix
### What changes were proposed in this pull request?
1. Remove the method `listPartitionIdentifiers()` from the `SupportsPartitionManagement` interface. The method lists partitions by ident prefix.
2. Rename `listPartitionByNames()` to `listPartitionIdentifiers()`.
3. Re-implement the default method `partitionExists()` using new method.

### Why are the changes needed?
Getting partitions by ident prefix only is not used, and it can be removed to improve code maintenance. Also this makes the `SupportsPartitionManagement` interface cleaner.

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

### How was this patch tested?
By running the affected test suites:
```
$ build/sbt "test:testOnly org.apache.spark.sql.connector.catalog.*"
```

Closes #30514 from MaxGekk/remove-listPartitionIdentifiers.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 14:05:49 +00:00
Max Gekk 0a612b6a40 [SPARK-33452][SQL] Support v2 SHOW PARTITIONS
### What changes were proposed in this pull request?
1. Remove V2 logical node `ShowPartitionsStatement `, and replace it by V2 `ShowPartitions`.
2. Implement V2 execution node `ShowPartitionsExec` similar to V1 `ShowPartitionsCommand`.

### Why are the changes needed?
To have feature parity with Datasource V1.

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

Before the change, `SHOW PARTITIONS` fails in V2 table catalogs with the exception:
```
org.apache.spark.sql.AnalysisException: SHOW PARTITIONS is only supported with v1 tables.
   at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog.org$apache$spark$sql$catalyst$analysis$ResolveSessionCatalog$$parseV1Table(ResolveSessionCatalog.scala:628)
   at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog$$anonfun$apply$1.applyOrElse(ResolveSessionCatalog.scala:466)
```

### How was this patch tested?
By running the following test suites:
1. Modified `ShowPartitionsParserSuite` where `ShowPartitionsStatement` is replaced by V2 `ShowPartitions`.
2. `v2.ShowPartitionsSuite`

Closes #30398 from MaxGekk/show-partitions-exec-v2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 13:45:53 +00:00
Wenchen Fan 5cfbdddefe [SPARK-33480][SQL] Support char/varchar type
### What changes were proposed in this pull request?

This PR adds the char/varchar type which is kind of a variant of string type:
1. Char type is fixed-length string. When comparing char type values, we need to pad the shorter one to the longer length.
2. Varchar type is string with a length limitation.

To implement the char/varchar semantic, this PR:
1. Do string length check when writing to char/varchar type columns.
2. Do string padding when reading char type columns. We don't do it at the writing side to save storage space.
3. Do string padding when comparing char type column with string literal or another char type column. (string literal is fixed length so should be treated as char type as well)

To simplify the implementation, this PR doesn't propagate char/varchar type info through functions/operators(e.g. `substring`). That said, a column can only be char/varchar type if it's a table column, not a derived column like `SELECT substring(col)`.

To be safe, this PR doesn't add char/varchar type to the query engine(expression input check, internal row framework, codegen framework, etc.). We will replace char/varchar type by string type with metadata (`Attribute.metadata` or `StructField.metadata`) that includes the original type string before it goes into the query engine. That said, the existing code will not see char/varchar type but only string type.

char/varchar type may come from several places:
1. v1 table from hive catalog.
2. v2 table from v2 catalog.
3. user-specified schema in `spark.read.schema` and `spark.readStream.schema`
4. `Column.cast`
5. schema string in places like `from_json`, pandas UDF, etc. These places use SQL parser which replaces char/varchar with string already, even before this PR.

This PR covers all the above cases, implements the length check and padding feature by looking at string type with special metadata.

### Why are the changes needed?

char and varchar are standard SQL types. varchar is widely used in other databases instead of string type.

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

For hive tables: now the table insertion fails if the value exceeds char/varchar length. Previously we truncate the value silently.

For other tables:
1. now char type is allowed.
2. now we have length check when inserting to varchar columns. Previously we write the value as it is.

### How was this patch tested?

new tests

Closes #30412 from cloud-fan/char.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 09:23:05 +00:00
gengjiaan b665d58819 [SPARK-28646][SQL] Fix bug of Count so as consistent with mainstream databases
### What changes were proposed in this pull request?
Currently, Spark allows calls to `count` even for non parameterless aggregate function. For example, the following query actually works:
`SELECT count() FROM tenk1;`
On the other hand, mainstream databases will throw an error.
**Oracle**
`> ORA-00909: invalid number of arguments`
**PgSQL**
`ERROR:  count(*) must be used to call a parameterless aggregate function`
**MySQL**
`> 1064 - You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near ')`

### Why are the changes needed?
Fix a bug so that consistent with mainstream databases.
There is an example query output with/without this fix.
`SELECT count() FROM testData;`
The output before this fix:
`0`
The output after this fix:
```
org.apache.spark.sql.AnalysisException
cannot resolve 'count()' due to data type mismatch: count requires at least one argument.; line 1 pos 7
```

### Does this PR introduce _any_ user-facing change?
Yes.
If not specify parameter for `count`, will throw an error.

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

Closes #30541 from beliefer/SPARK-28646.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-30 17:04:38 +09:00
Terry Kim 0fd9f57dd4 [SPARK-33448][SQL] Support CACHE/UNCACHE TABLE commands for v2 tables
### What changes were proposed in this pull request?

This PR proposes to support `CHACHE/UNCACHE TABLE` commands for v2 tables.

In addtion, this PR proposes to migrate `CACHE/UNCACHE 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).

### Why are the changes needed?

To support `CACHE/UNCACHE TABLE` commands for v2 tables.

Note that `CACHE/UNCACHE TABLE` for v1 tables/views go through `SparkSession.table` to resolve identifier, which resolves temp views first, so there is no change in the behavior by moving to the new framework.

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

Yes. Now the user can run `CACHE/UNCACHE TABLE` commands on v2 tables.

### How was this patch tested?

Added/updated existing tests.

Closes #30403 from imback82/cache_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 05:37:10 +00:00
Kent Yao 2da72593c1 [SPARK-32976][SQL] Support column list in INSERT statement
### What changes were proposed in this pull request?

#### JIRA expectations
```
   INSERT currently does not support named column lists.

   INSERT INTO <table> (col1, col2,…) VALUES( 'val1', 'val2', … )
   Note, we assume the column list contains all the column names. Issue an exception if the list is not complete. The column order could be different from the column order defined in the table definition.
```
#### implemetations
In this PR, we add a column list  as an optional part to the `INSERT OVERWRITE/INTO` statements:
```
  /**
   * {{{
   *   INSERT OVERWRITE TABLE tableIdentifier [partitionSpec [IF NOT EXISTS]]? [identifierList] ...
   *   INSERT INTO [TABLE] tableIdentifier [partitionSpec]  [identifierList] ...
   * }}}
   */
```
The column list represents all expected columns with an explicit order that you want to insert to the target table. **Particularly**,  we assume the column list contains all the column names in the current implementation, it will fail when the list is incomplete.

In **Analyzer**, we add a code path to resolve the column list in the `ResolveOutputRelation` rule before it is transformed to v1 or v2 command. It will fail here if the list has any field that not belongs to the target table.

Then, for v2 command, e.g. `AppendData`, we use the resolved column list and output of the target table to resolve the output of the source query `ResolveOutputRelation` rule. If the list has duplicated columns, we fail. If the list is not empty but the list size does not match the target table, we fail. If no other exceptions occur, we use the column list to map the output of the source query to the output of the target table.  The column list will be set to Nil and it will not hit the rule again after it is resolved.

for v1 command, those all happen in the `PreprocessTableInsertion` rule

### Why are the changes needed?
 new feature support

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

yes, insert into/overwrite table support specify column list
### How was this patch tested?

new tests

Closes #29893 from yaooqinn/SPARK-32976.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 05:23:23 +00:00
Chao Sun feda7299e3 [SPARK-33567][SQL] DSv2: Use callback instead of passing Spark session and v2 relation for refreshing cache
### What changes were proposed in this pull request?

This replaces Spark session and `DataSourceV2Relation` in V2 write plans by replacing them with a callback `afterWrite`.

### Why are the changes needed?

Per discussion in #30429, it's better to not pass Spark session and `DataSourceV2Relation` through Spark plans. Instead we can use a callback which makes the interface cleaner.

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

No

### How was this patch tested?

N/A

Closes #30491 from sunchao/SPARK-33492-followup.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 04:50:50 +00:00
Max Gekk a088a801ed
[SPARK-33585][SQL][DOCS] Fix the comment for SQLContext.tables() and mention the database column
### What changes were proposed in this pull request?
Change the comments for `SQLContext.tables()` to "The returned DataFrame has three columns, database, tableName and isTemporary".

### Why are the changes needed?
Currently, the comment mentions only 2 columns but `tables()` returns 3 columns actually:
```scala
scala> spark.range(10).createOrReplaceTempView("view1")
scala> val tables = spark.sqlContext.tables()
tables: org.apache.spark.sql.DataFrame = [database: string, tableName: string ... 1 more field]

scala> tables.printSchema
root
 |-- database: string (nullable = false)
 |-- tableName: string (nullable = false)
 |-- isTemporary: boolean (nullable = false)

scala> tables.show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
| default|       t1|      false|
| default|       t2|      false|
| default|      ymd|      false|
|        |    view1|       true|
+--------+---------+-----------+
```

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

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

Closes #30526 from MaxGekk/sqlcontext-tables-doc.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-29 12:18:07 -08:00
Max Gekk 0054fc937f
[SPARK-33588][SQL] Respect the spark.sql.caseSensitive config while resolving partition spec in v1 SHOW TABLE EXTENDED
### What changes were proposed in this pull request?
Perform partition spec normalization in `ShowTablesCommand` according to the table schema before getting partitions from the catalog. The normalization via `PartitioningUtils.normalizePartitionSpec()` adjusts the column names in partition specification, w.r.t. the real partition column names and case sensitivity.

### Why are the changes needed?
Even when `spark.sql.caseSensitive` is `false` which is the default value, v1 `SHOW TABLE EXTENDED` is case sensitive:
```sql
spark-sql> CREATE TABLE tbl1 (price int, qty int, year int, month int)
         > USING parquet
         > partitioned by (year, month);
spark-sql> INSERT INTO tbl1 PARTITION(year = 2015, month = 1) SELECT 1, 1;
spark-sql> SHOW TABLE EXTENDED LIKE 'tbl1' PARTITION(YEAR = 2015, Month = 1);
Error in query: Partition spec is invalid. The spec (YEAR, Month) must match the partition spec (year, month) defined in table '`default`.`tbl1`';
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the `SHOW TABLE EXTENDED` command respects the SQL config. And for example above, it returns correct result:
```sql
spark-sql> SHOW TABLE EXTENDED LIKE 'tbl1' PARTITION(YEAR = 2015, Month = 1);
default	tbl1	false	Partition Values: [year=2015, month=1]
Location: file:/Users/maximgekk/spark-warehouse/tbl1/year=2015/month=1
Serde Library: org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
Storage Properties: [serialization.format=1, path=file:/Users/maximgekk/spark-warehouse/tbl1]
Partition Parameters: {transient_lastDdlTime=1606595118, totalSize=623, numFiles=1}
Created Time: Sat Nov 28 23:25:18 MSK 2020
Last Access: UNKNOWN
Partition Statistics: 623 bytes
```

### How was this patch tested?
By running the modified test suite `v1/ShowTablesSuite`

Closes #30529 from MaxGekk/show-table-case-sensitive-spec.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-29 12:10:16 -08:00
Max Gekk bfe9380ba2
[MINOR][SQL] Remove getTables() from r.SQLUtils
### What changes were proposed in this pull request?
Remove the unused method `getTables()` from `r.SQLUtils`. The method was used before the changes https://github.com/apache/spark/pull/17483 but R's `tables.default` was rewritten using `listTables()`: https://github.com/apache/spark/pull/17483/files#diff-2c01472a7bcb1d318244afcd621d726e00d36cd15dffe7e44fa96c54fce4cd9aR220-R223

### Why are the changes needed?
To improve code maintenance, and remove the dead code.

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

### How was this patch tested?
By R tests.

Closes #30527 from MaxGekk/remove-getTables-in-r-SQLUtils.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-28 16:58:40 -08:00
luluorta 35ded12fc6 [SPARK-33141][SQL] Capture SQL configs when creating permanent views
### What changes were proposed in this pull request?
This PR makes CreateViewCommand/AlterViewAsCommand capturing runtime SQL configs and store them as view properties. These configs will be applied during the parsing and analysis phases of the view resolution. Users can set `spark.sql.legacy.useCurrentConfigsForView` to `true` to restore the behavior before.

### Why are the changes needed?
This PR is a sub-task of [SPARK-33138](https://issues.apache.org/jira/browse/SPARK-33138) that proposes to unify temp view and permanent view behaviors. This PR makes permanent views mimicking the temp view behavior that "fixes" view semantic by directly storing resolved LogicalPlan. For example, if a user uses spark 2.4 to create a view that contains null values from division-by-zero expressions, she may not want that other users' queries which reference her view throw exceptions when running on spark 3.x with ansi mode on.

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

### How was this patch tested?
added UT + existing UTs (improved)

Closes #30289 from luluorta/SPARK-33141.

Authored-by: luluorta <luluorta@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-27 13:32:25 +00:00
xuewei.linxuewei b9f2f78de5 [SPARK-33498][SQL] Datetime parsing should fail if the input string can't be parsed, or the pattern string is invalid
### What changes were proposed in this pull request?

Datetime parsing should fail if the input string can't be parsed, or the pattern string is invalid, when ANSI mode is enable. This patch should update GetTimeStamp, UnixTimeStamp, ToUnixTimeStamp and Cast.

### 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 #30442 from leanken/leanken-SPARK-33498.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-27 13:24:11 +00:00
gengjiaan e43255051c [SPARK-28645][SQL] ParseException is thrown when the window is redefined
### What changes were proposed in this pull request?
Currently in Spark one could redefine a window. For instance:

`select count(*) OVER w FROM tenk1 WINDOW w AS (ORDER BY unique1), w AS (ORDER BY unique1);`
The window `w` is defined two times. In PgSQL, on the other hand, a thrown will happen:

`ERROR:  window "w" is already defined`

### Why are the changes needed?
The current implement gives the following window definitions a higher priority. But it wasn't Spark's intention and users can't know from any document of Spark.
This PR fixes the bug.

### Does this PR introduce _any_ user-facing change?
Yes.
There is an example query output with/without this fix.
```
SELECT
    employee_name,
    salary,
    first_value(employee_name) OVER w highest_salary,
    nth_value(employee_name, 2) OVER w second_highest_salary
FROM
    basic_pays
WINDOW
    w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING),
    w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING)
ORDER BY salary DESC
```
The output before this fix:
```
Larry Bott	11798	Larry Bott	Gerard Bondur
Gerard Bondur	11472	Larry Bott	Gerard Bondur
Pamela Castillo	11303	Larry Bott	Gerard Bondur
Barry Jones	10586	Larry Bott	Gerard Bondur
George Vanauf	10563	Larry Bott	Gerard Bondur
Loui Bondur	10449	Larry Bott	Gerard Bondur
Mary Patterson	9998	Larry Bott	Gerard Bondur
Steve Patterson	9441	Larry Bott	Gerard Bondur
Julie Firrelli	9181	Larry Bott	Gerard Bondur
Jeff Firrelli	8992	Larry Bott	Gerard Bondur
William Patterson	8870	Larry Bott	Gerard Bondur
Diane Murphy	8435	Larry Bott	Gerard Bondur
Leslie Jennings	8113	Larry Bott	Gerard Bondur
Gerard Hernandez	6949	Larry Bott	Gerard Bondur
Foon Yue Tseng	6660	Larry Bott	Gerard Bondur
Anthony Bow	6627	Larry Bott	Gerard Bondur
Leslie Thompson	5186	Larry Bott	Gerard Bondur
```
The output after this fix:
```
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException

The definition of window 'w' is repetitive(line 8, pos 0)
```

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

Closes #30512 from beliefer/SPARK-28645.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-27 10:27:08 +00:00
Terry Kim 2c41d9d8fa [SPARK-33522][SQL] Improve exception messages while handling UnresolvedTableOrView
### What changes were proposed in this pull request?

This PR proposes to improve the exception messages while `UnresolvedTableOrView` is handled based on this suggestion: https://github.com/apache/spark/pull/30321#discussion_r521127001.

Currently, when an identifier is resolved to a temp view when a table/permanent view is expected, the following exception message is displayed (e.g., for `SHOW CREATE TABLE`):
```
t is a temp view not table or permanent view.
```
After this PR, the message will be:
```
t is a temp view. 'SHOW CREATE TABLE' expects a table or permanent view.
```

Also, if an identifier is not resolved, the following exception message is currently used:
```
Table or view not found: t
```
After this PR, the message will be:
```
Table or permanent view not found for 'SHOW CREATE TABLE': t
```
or
```
Table or view not found for 'ANALYZE TABLE ... FOR COLUMNS ...': t
```

### Why are the changes needed?

To improve the exception message.

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

Yes, the exception message will be changed as described above.

### How was this patch tested?

Updated existing tests.

Closes #30475 from imback82/unresolved_table_or_view.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-27 10:16:56 +00:00
Terry Kim 8792280a73 [SPARK-33575][SQL] Fix misleading exception for "ANALYZE TABLE ... FOR COLUMNS" on temporary views
### What changes were proposed in this pull request?

This PR proposes to fix the exception message for `ANALYZE TABLE ... FOR COLUMNS` on temporary views.

The current behavior throws `NoSuchTableException` even if the temporary view exists:
```
sql("CREATE TEMP VIEW t AS SELECT 1 AS id")
sql("ANALYZE TABLE t COMPUTE STATISTICS FOR COLUMNS id")
org.apache.spark.sql.catalyst.analysis.NoSuchTableException: Table or view 't' not found in database 'db';
  at org.apache.spark.sql.execution.command.AnalyzeColumnCommand.analyzeColumnInTempView(AnalyzeColumnCommand.scala:76)
  at org.apache.spark.sql.execution.command.AnalyzeColumnCommand.run(AnalyzeColumnCommand.scala:54)
```

After this PR, more reasonable exception is thrown:
```
org.apache.spark.sql.AnalysisException: Temporary view `testView` is not cached for analyzing columns.;
[info]   at org.apache.spark.sql.execution.command.AnalyzeColumnCommand.analyzeColumnInTempView(AnalyzeColumnCommand.scala:74)
[info]   at org.apache.spark.sql.execution.command.AnalyzeColumnCommand.run(AnalyzeColumnCommand.scala:54)
```

### Why are the changes needed?

To fix a misleading exception.

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

Yes, the exception thrown is changed as shown above.

### How was this patch tested?

Updated existing test.

Closes #30519 from imback82/analyze_table_message.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-27 07:08:24 +00:00
yangjie01 433ae9064f [SPARK-33566][CORE][SQL][SS][PYTHON] Make unescapedQuoteHandling option configurable when read CSV
### What changes were proposed in this pull request?
There are some differences between Spark CSV, opencsv and commons-csv, the typical case are described in SPARK-33566, When there are both unescaped quotes and unescaped qualifier in value,  the results of parsing are different.

The reason for the difference is Spark use `STOP_AT_DELIMITER` as default `UnescapedQuoteHandling` to build `CsvParser` and it not configurable.

On the other hand, opencsv and commons-csv use the parsing mechanism similar to `STOP_AT_CLOSING_QUOTE ` by default.

So this pr make `unescapedQuoteHandling` option configurable to get the same parsing result as opencsv and commons-csv.

### Why are the changes needed?
Make unescapedQuoteHandling option configurable when read CSV to make parsing more flexible。

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

### How was this patch tested?

- Pass the Jenkins or GitHub Action

- Add a new case similar to that described in SPARK-33566

Closes #30518 from LuciferYang/SPARK-33566.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-27 15:47:39 +09:00
Maryann Xue dfa3978d91 [SPARK-33551][SQL] Do not use custom shuffle reader for repartition
### What changes were proposed in this pull request?

This PR fixes an AQE issue where local shuffle reader, partition coalescing, or skew join optimization can be mistakenly applied to a shuffle introduced by repartition or a regular shuffle that logically replaces a repartition shuffle.
The proposed solution checks for the presence of any repartition shuffle and filters out not applicable optimization rules for the final stage in an AQE plan.

### Why are the changes needed?

Without the change, the output of a repartition query may not be correct.

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

No.

### How was this patch tested?

Added UT.

Closes #30494 from maryannxue/csr-repartition.

Authored-by: Maryann Xue <maryann.xue@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-11-25 19:32:22 -08:00
Liang-Chi Hsieh fb7b870214 [SPARK-33523][SQL][TEST][FOLLOWUP] Fix benchmark case name in SubExprEliminationBenchmark
### What changes were proposed in this pull request?

Fix the wrong benchmark case name.

### Why are the changes needed?

The last commit to refactor the benchmark code missed a change of case name.

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

No, dev only.

### How was this patch tested?

Unit test.

Closes #30505 from viirya/SPARK-33523-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-25 15:22:47 -08:00
Dongjoon Hyun 7cf6a6f996 [SPARK-31257][SPARK-33561][SQL][FOLLOWUP] Fix Scala 2.13 compilation
### What changes were proposed in this pull request?

This PR is a follow-up to fix Scala 2.13 compilation.

### Why are the changes needed?

To support Scala 2.13 in Apache Spark 3.1.

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

No.

### How was this patch tested?

Pass the GitHub Action Scala 2.13 compilation job.

Closes #30502 from dongjoon-hyun/SPARK-31257.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-25 09:57:46 -08:00
Liang-Chi Hsieh 9643eab53e [SPARK-33540][SQL] Subexpression elimination for interpreted predicate
### What changes were proposed in this pull request?

This patch proposes to support subexpression elimination for interpreted predicate.

### Why are the changes needed?

Similar to interpreted projection, there are use cases when codegen predicate is not able to work, e.g. too complex schema, non-codegen expression, etc. When there are frequently occurring expressions (subexpressions) among predicate expression, the performance is quite bad as we need to re-compute same expressions. We should be able to support subexpression elimination for interpreted predicate like interpreted projection.

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

No, this doesn't change user behavior.

### How was this patch tested?

Unit test and benchmark.

Closes #30497 from viirya/SPARK-33540.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-25 08:55:39 -08:00
Ryan Blue 6f68ccf532 [SPARK-31257][SPARK-33561][SQL] Unify create table syntax
### What changes were proposed in this pull request?

* Unify the create table syntax in the parser by merging Hive and DataSource clauses
* Add `SerdeInfo` and `external` boolean to statement plans and update AstBuilder to produce them
* Add conversion from create statement plan to v1 create plans in ResolveSessionCatalog
* Support new statement clauses in ResolveCatalogs conversion to v2 create plans
* Remove SparkSqlParser rules for Hive syntax
* Add "option." namespace to distinguish SERDEPROPERTIES and OPTIONS in table properties

### Why are the changes needed?

* Current behavior is confusing.
* A way to pass the Hive create options to DSv2 is needed for a Hive source.

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

Not by default, but v2 sources will be able to handle STORED AS and other Hive clauses.

### How was this patch tested?

Existing tests validate there are no behavior changes.

Update unit tests for using a statement plan for Hive create syntax:
* Move create tests from spark-sql DDLParserSuite into PlanResolutionSuite
* Add parser tests to spark-catalyst DDLParserSuite

Closes #28026 from rdblue/unify-create-table.

Lead-authored-by: Ryan Blue <blue@apache.org>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-25 15:09:02 +00:00
duripeng 7c59aeeef4 [SPARK-27194][SPARK-29302][SQL] Fix commit collision in dynamic partition overwrite mode
### What changes were proposed in this pull request?

When using dynamic partition overwrite, each task has its working dir under staging dir like `stagingDir/.spark-staging-{jobId}`, each task commits to `outputPath/.spark-staging-{jobId}/{partitionId}/part-{taskId}-{jobId}{ext}`.
When speculation enable, multiple task attempts would be setup for one task, **they have same task id and they would commit to same file concurrently**. Due to host done or node preemption, the partly-committed files aren't cleaned up, a FileAlreadyExistsException would be raised in this situation, resulting in job failure.

I don't try to change task commit process for dynamic partition overwrite, like adding attempt id to task working dir for each attempts and committing to final output dir via a new outputCommitCoordinator, here is reason:

1. `FileOutputCommitter` already has commit coordinator for each task attempts, we can leverage it rather than build a new one.
2. To say the least, we implement a coordinator solving task attempts commit conflict, suppose a severe case, application master failover, tasks with same attempt id and same task id would commit to same files, the `FileAlreadyExistsException` risk still exists

In this pr, I leverage FileOutputCommitter to solve the problem:

1. when initing a write job description, set `outputPath/.spark-staging-{jobId}` as the output dir
2. each task attempt writes output to `outputPath/.spark-staging-{jobId}/_temporary/${appAttemptId}/_temporary/${taskAttemptId}/{partitionId}/part-{taskId}-{jobId}{ext}`
3. leverage `FileOutputCommitter` coordinator, write job firstly commits output to `outputPath/.spark-staging-{jobId}/{partitionId}`
4. for dynamic partition overwrite, write job finally move `outputPath/.spark-staging-{jobId}/{partitionId}` to `outputPath/{partitionId}`

### Why are the changes needed?

Without this pr, dynamic partition overwrite would fail

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

No.

### How was this patch tested?

added UT.

Closes #29000 from WinkerDu/master-fix-dynamic-partition-multi-commit.

Authored-by: duripeng <duripeng@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-25 12:50:21 +00:00
Kousuke Saruta c3ce9701b4 [SPARK-33533][SQL] Fix the regression bug that ConnectionProviders don't consider case-sensitivity for properties
### What changes were proposed in this pull request?

This PR fixes an issue that `BasicConnectionProvider` doesn't consider case-sensitivity for properties.
For example, the property `oracle.jdbc.mapDateToTimestamp` should be considered case-sensitivity but it is not considered.

### Why are the changes needed?

This is a bug introduced by #29024 .
Caused by this issue, `OracleIntegrationSuite` doesn't pass.

```
[info] - SPARK-16625: General data types to be mapped to Oracle *** FAILED *** (32 seconds, 129 milliseconds)
[info]   types.apply(9).equals(org.apache.spark.sql.types.DateType) was false (OracleIntegrationSuite.scala:238)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
[info]   at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
[info]   at org.scalatest.Assertions$.newAssertionFailedException(Assertions.scala:1231)
[info]   at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:1295)
[info]   at org.apache.spark.sql.jdbc.OracleIntegrationSuite.$anonfun$new$4(OracleIntegrationSuite.scala:238)
[info]   at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
[info]   at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
[info]   at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:22)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:20)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:190)
[info]   at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:176)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:188)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:200)
[info]   at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:200)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:182)
[info]   at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:61)
[info]   at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
[info]   at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
[info]   at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:61)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:233)
[info]   at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
[info]   at scala.collection.immutable.List.foreach(List.scala:392)
[info]   at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
[info]   at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396)
[info]   at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:233)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:232)
[info]   at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1563)
[info]   at org.scalatest.Suite.run(Suite.scala:1112)
[info]   at org.scalatest.Suite.run$(Suite.scala:1094)
[info]   at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1563)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:237)
[info]   at org.scalatest.SuperEngine.runImpl(Engine.scala:535)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:237)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:236)
[info]   at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:61)
[info]   at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
[info]   at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
[info]   at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
[info]   at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:61)
[info]   at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:318)
[info]   at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:513)
[info]   at sbt.ForkMain$Run.lambda$runTest$1(ForkMain.java:413)
[info]   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
[info]   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
[info]   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[info]   at java.lang.Thread.run(Thread.java:748)
```

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

No.

### How was this patch tested?

With this change, I confirmed that `OracleIntegrationSuite` passes with the following command.
```
$ git clone https://github.com/oracle/docker-images.git
$ cd docker-images/OracleDatabase/SingleInstance/dockerfiles
$ ./buildDockerImage.sh -v 18.4.0 -x
$ ORACLE_DOCKER_IMAGE_NAME=oracle/database:18.4.0-xe build/sbt  -Pdocker-integration-tests -Phive -Phive-thriftserver "testOnly org.apache.spark.sql.jdbc.OracleIntegrationSuite"
```

Closes #30485 from sarutak/fix-oracle-integration-suite.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-24 20:18:45 -08:00
Jungtaek Lim (HeartSaVioR) edab094dda [SPARK-33224][SS][WEBUI] Add watermark gap information into SS UI page
### What changes were proposed in this pull request?

This PR proposes to add the watermark gap information in SS UI page. Please refer below screenshots to see what we'd like to show in UI.

![Screen Shot 2020-11-19 at 6 56 38 PM](https://user-images.githubusercontent.com/1317309/99669306-3532d080-2ab2-11eb-9a93-03d2c6a54948.png)

Please note that this PR doesn't plot the watermark value - knowing the gap between actual wall clock and watermark looks more useful than the absolute value.

### Why are the changes needed?

Watermark is the one of major metrics the end users need to track for stateful queries. Watermark defines "when" the output will be emitted for append mode, hence knowing how much gap between wall clock and watermark (input data) is very helpful to make expectation of the output.

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

Yes, SS UI query page will contain the watermark gap information.

### How was this patch tested?

Basic UT added. Manually tested with two queries:

> simple case

You'll see consistent watermark gap with (15 seconds + a) = 10 seconds are from delay in watermark definition, 5 seconds are trigger interval.

```
import org.apache.spark.sql.streaming.Trigger

spark.conf.set("spark.sql.shuffle.partitions", "10")

val query = spark
  .readStream
  .format("rate")
  .option("rowsPerSecond", 1000)
  .option("rampUpTime", "10s")
  .load()
  .selectExpr("timestamp", "mod(value, 100) as mod", "value")
  .withWatermark("timestamp", "10 seconds")
  .groupBy(window($"timestamp", "1 minute", "10 seconds"), $"mod")
  .agg(max("value").as("max_value"), min("value").as("min_value"), avg("value").as("avg_value"))
  .writeStream
  .format("console")
  .trigger(Trigger.ProcessingTime("5 seconds"))
  .outputMode("append")
  .start()

query.awaitTermination()
```

![Screen Shot 2020-11-19 at 7 00 21 PM](https://user-images.githubusercontent.com/1317309/99669049-dbcaa180-2ab1-11eb-8789-10b35857dda0.png)

> complicated case

This randomizes the timestamp, hence producing random watermark gap. This won't be smaller than 15 seconds as I described earlier.

```
import org.apache.spark.sql.streaming.Trigger

spark.conf.set("spark.sql.shuffle.partitions", "10")

val query = spark
  .readStream
  .format("rate")
  .option("rowsPerSecond", 1000)
  .option("rampUpTime", "10s")
  .load()
  .selectExpr("*", "CAST(CAST(timestamp AS BIGINT) - CAST((RAND() * 100000) AS BIGINT) AS TIMESTAMP) AS tsMod")
  .selectExpr("tsMod", "mod(value, 100) as mod", "value")
  .withWatermark("tsMod", "10 seconds")
  .groupBy(window($"tsMod", "1 minute", "10 seconds"), $"mod")
  .agg(max("value").as("max_value"), min("value").as("min_value"), avg("value").as("avg_value"))
  .writeStream
  .format("console")
  .trigger(Trigger.ProcessingTime("5 seconds"))
  .outputMode("append")
  .start()

query.awaitTermination()
```

![Screen Shot 2020-11-19 at 6 56 47 PM](https://user-images.githubusercontent.com/1317309/99669029-d5d4c080-2ab1-11eb-9c63-d05b3e1ab391.png)

Closes #30427 from HeartSaVioR/SPARK-33224.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-25 13:12:20 +09:00
Terry Kim b7f034d8dc [SPARK-33543][SQL] Migrate SHOW COLUMNS command to use UnresolvedTableOrView to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `SHOW 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 `SHOW COLUMNS` is not yet supported for v2 tables.

### Why are the changes needed?

To use `UnresolvedTableOrView` for table/view resolution. Note that `ShowColumnsCommand` internally resolves to a temp view first, so there is no resolution behavior change with this PR.

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

No.

### How was this patch tested?

Updated existing tests.

Closes #30490 from imback82/show_columns.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-25 03:04:04 +00:00
Wenchen Fan d1b4f06179 [SPARK-33494][SQL][AQE] Do not use local shuffle reader for repartition
### What changes were proposed in this pull request?

This PR updates `ShuffleExchangeExec` to carry more information about how much we can change the partitioning. For `repartition(col)`, we should preserve the user-specified partitioning and don't apply the AQE local shuffle reader.

### Why are the changes needed?

Similar to `repartition(number, col)`, we should respect the user-specified partitioning.

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

No

### How was this patch tested?

a new test

Closes #30432 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-25 02:02:32 +00:00
Gabor Somogyi 95b6dabc33 [SPARK-33287][SS][UI] Expose state custom metrics information on SS UI
### What changes were proposed in this pull request?
Structured Streaming UI is not containing state custom metrics information. In this PR I've added it.

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

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

### How was this patch tested?
Existing unit tests + manual test.
```
#Compile Spark
echo "spark.sql.streaming.ui.enabledCustomMetricList stateOnCurrentVersionSizeBytes" >> conf/spark-defaults.conf
sbin/start-master.sh
sbin/start-worker.sh spark://gsomogyi-MBP16:7077
./bin/spark-submit --master spark://gsomogyi-MBP16:7077 --deploy-mode client --class com.spark.Main ../spark-test/target/spark-test-1.0-SNAPSHOT-jar-with-dependencies.jar
```
<img width="1119" alt="Screenshot 2020-11-18 at 12 45 36" src="https://user-images.githubusercontent.com/18561820/99527506-2f979680-299d-11eb-9187-4ae7fbd2596a.png">

Closes #30336 from gaborgsomogyi/SPARK-33287.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-25 07:38:45 +09:00
Terry Kim fdd6c73b3c [SPARK-33514][SQL] Migrate TRUNCATE TABLE command to use UnresolvedTable to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `TRUNCATE TABLE` 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 `TRUNCATE TABLE` works only with v1 tables, 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 using csv AS SELECT 1")
sql("USE db")
sql("TRUNCATE TABLE t") // Succeeds
```
With this PR, `TRUNCATE TABLE` 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$42(Analyzer.scala:866)

```
, which is expected since temporary view is resolved first and `TRUNCATE TABLE` doesn't support a temporary view.

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

After this PR, `TRUNCATE TABLE` 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 #30457 from imback82/truncate_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-24 11:06:39 +00:00
Max Gekk a6555ee596 [SPARK-33521][SQL] Universal type conversion in resolving V2 partition specs
### What changes were proposed in this pull request?
In the PR, I propose to changes the resolver of partition specs used in V2 `ALTER TABLE .. ADD/DROP PARTITION` (at the moment), and re-use `CAST` in conversion partition values to desired types according to the partition schema.

### Why are the changes needed?
Currently, the resolver of V2 partition specs supports just a few types: 23e9920b39/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala (L72), and fails on other types like date/timestamp.

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

### How was this patch tested?
By running `AlterTablePartitionV2SQLSuite`

Closes #30474 from MaxGekk/dsv2-partition-value-types.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-24 08:04:21 +00:00
Liang-Chi Hsieh f35e28fea5 [SPARK-33523][SQL][TEST] Add predicate related benchmark to SubExprEliminationBenchmark
### What changes were proposed in this pull request?

This patch adds predicate related benchmark to `SubExprEliminationBenchmark`.

### Why are the changes needed?

We should have a benchmark for subexpression elimination of predicate.

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

No, dev only.

### How was this patch tested?

Run benchmark locally.

Closes #30476 from viirya/SPARK-33523.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-24 13:30:06 +09:00
Dongjoon Hyun 8380e00419
[SPARK-33524][SQL][TESTS] Change InMemoryTable not to use Tuple.hashCode for BucketTransform
### What changes were proposed in this pull request?

This PR aims to change `InMemoryTable` not to use `Tuple.hashCode` for `BucketTransform`.

### Why are the changes needed?

SPARK-32168 made `InMemoryTable` to handle `BucketTransform` as a hash of `Tuple` which is dependents on Scala versions.
- https://github.com/apache/spark/blob/master/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala#L159

**Scala 2.12.10**
```scala
$ bin/scala
Welcome to Scala 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_272).
Type in expressions for evaluation. Or try :help.

scala> (1, 1).hashCode
res0: Int = -2074071657
```

**Scala 2.13.3**
```scala
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_272).
Type in expressions for evaluation. Or try :help.

scala> (1, 1).hashCode
val res0: Int = -1669302457
```

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

Yes. This is a correctness issue.

### How was this patch tested?

Pass the UT with both Scala 2.12/2.13.

Closes #30477 from dongjoon-hyun/SPARK-33524.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-23 19:35:58 -08:00
Dongjoon Hyun 3ce4ab545b
[SPARK-33513][BUILD] Upgrade to Scala 2.13.4 to improve exhaustivity
### What changes were proposed in this pull request?

This PR aims the followings.
1. Upgrade from Scala 2.13.3 to 2.13.4 for Apache Spark 3.1
2. Fix exhaustivity issues in both Scala 2.12/2.13 (Scala 2.13.4 requires this for compilation.)
3. Enforce the improved exhaustive check by using the existing Scala 2.13 GitHub Action compilation job.

### Why are the changes needed?

Scala 2.13.4 is a maintenance release for 2.13 line and improves JDK 15 support.
- https://github.com/scala/scala/releases/tag/v2.13.4

Also, it improves exhaustivity check.
- https://github.com/scala/scala/pull/9140 (Check exhaustivity of pattern matches with "if" guards and custom extractors)
- https://github.com/scala/scala/pull/9147 (Check all bindings exhaustively, e.g. tuples components)

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

Yep. Although it's a maintenance version change, it's a Scala version change.

### How was this patch tested?

Pass the CIs and do the manual testing.
- Scala 2.12 CI jobs(GitHub Action/Jenkins UT/Jenkins K8s IT) to check the validity of code change.
- Scala 2.13 Compilation job to check the compilation

Closes #30455 from dongjoon-hyun/SCALA_3.13.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-23 16:28:43 -08:00
Max Gekk 23e9920b39 [SPARK-33511][SQL] Respect case sensitivity while resolving V2 partition specs
### What changes were proposed in this pull request?
1. Pre-process partition specs in `ResolvePartitionSpec`, and convert partition names according to the partition schema and the SQL config `spark.sql.caseSensitive`. In the PR, I propose to invoke `normalizePartitionSpec` for that. The function is used in DSv1 commands, so, the behavior will be similar to DSv1.
2. Move `normalizePartitionSpec()` from `sql/core/.../datasources/PartitioningUtils` to `sql/catalyst/.../util/PartitioningUtils` to use it in Catalyst's rule `ResolvePartitionSpec`

### Why are the changes needed?
DSv1 commands like `ALTER TABLE .. ADD PARTITION` and `ALTER TABLE .. DROP PARTITION` respect the SQL config `spark.sql.caseSensitive` while resolving partition specs. For example:
```sql
spark-sql> CREATE TABLE tbl1 (id bigint, data string) USING parquet PARTITIONED BY (id);
spark-sql> ALTER TABLE tbl1 ADD PARTITION (ID=1);
spark-sql> SHOW PARTITIONS tbl1;
id=1
```
The same command fails on V2 Table catalog with error:
```
AnalysisException: Partition key ID not exists
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, partition spec resolution works as for DSv1 (without the exception showed above).

### How was this patch tested?
By running `AlterTablePartitionV2SQLSuite`.

Closes #30454 from MaxGekk/partition-spec-case-sensitivity.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-23 09:00:41 +00:00