Commit graph

4844 commits

Author SHA1 Message Date
HyukjinKwon df8d3f1bf7 [SPARK-33544][SQL][FOLLOW-UP] Rename NoSideEffect to NoThrow and clarify the documentation more
### What changes were proposed in this pull request?

This PR is a followup of https://github.com/apache/spark/pull/30504. It proposes:

- Rename `NoSideEffect` to `NoThrow`, and use `Expression.deterministic` together where it is used.
- Clarify, in the docs in the expressions, that it means they don't throw exceptions

### Why are the changes needed?

`NoSideEffect` virtually means that `Expression.eval` does not throw an exception, and the expressions are deterministic.
It's best to be explicit so `NoThrow` was proposed -  I looked if there's a similar name to represent this concept and borrowed the name of [nothrow](https://clang.llvm.org/docs/AttributeReference.html#nothrow).
For determinism, we already have a way to note it under `Expression.deterministic`.

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

No

### How was this patch tested?

Manually ran the existing unittests written.

Closes #30570 from HyukjinKwon/SPARK-33544.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-02 16:03:08 +00:00
Dongjoon Hyun 290aa02179 [SPARK-33618][CORE] Use hadoop-client instead of hadoop-client-api to make hadoop-aws work
### What changes were proposed in this pull request?

This reverts commit SPARK-33212 (cb3fa6c936) mostly with three exceptions:
1. `SparkSubmitUtils` was updated recently by SPARK-33580
2. `resource-managers/yarn/pom.xml` was updated recently by SPARK-33104 to add `hadoop-yarn-server-resourcemanager` test dependency.
3. Adjust `com.fasterxml.jackson.module:jackson-module-jaxb-annotations` dependency in K8s module which is updated recently by SPARK-33471.

### Why are the changes needed?

According to [HADOOP-16080](https://issues.apache.org/jira/browse/HADOOP-16080) since Apache Hadoop 3.1.1, `hadoop-aws` doesn't work with `hadoop-client-api`. It fails at write operation like the following.

**1. Spark distribution with `-Phadoop-cloud`**

```scala
$ bin/spark-shell --conf spark.hadoop.fs.s3a.access.key=$AWS_ACCESS_KEY_ID --conf spark.hadoop.fs.s3a.secret.key=$AWS_SECRET_ACCESS_KEY
20/11/30 23:01:24 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Spark context available as 'sc' (master = local[*], app id = local-1606806088715).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.1.0-SNAPSHOT
      /_/

Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_272)
Type in expressions to have them evaluated.
Type :help for more information.

scala> spark.read.parquet("s3a://dongjoon/users.parquet").show
20/11/30 23:01:34 WARN MetricsConfig: Cannot locate configuration: tried hadoop-metrics2-s3a-file-system.properties,hadoop-metrics2.properties
+------+--------------+----------------+
|  name|favorite_color|favorite_numbers|
+------+--------------+----------------+
|Alyssa|          null|  [3, 9, 15, 20]|
|   Ben|           red|              []|
+------+--------------+----------------+

scala> Seq(1).toDF.write.parquet("s3a://dongjoon/out.parquet")
20/11/30 23:02:14 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 2)/ 1]
java.lang.NoSuchMethodError: org.apache.hadoop.util.SemaphoredDelegatingExecutor.<init>(Lcom/google/common/util/concurrent/ListeningExecutorService;IZ)V
```

**2. Spark distribution without `-Phadoop-cloud`**
```scala
$ bin/spark-shell --conf spark.hadoop.fs.s3a.access.key=$AWS_ACCESS_KEY_ID --conf spark.hadoop.fs.s3a.secret.key=$AWS_SECRET_ACCESS_KEY -c spark.eventLog.enabled=true -c spark.eventLog.dir=s3a://dongjoon/spark-events/ --packages org.apache.hadoop:hadoop-aws:3.2.0,org.apache.hadoop:hadoop-common:3.2.0
...
java.lang.NoSuchMethodError: org.apache.hadoop.util.SemaphoredDelegatingExecutor.<init>(Lcom/google/common/util/concurrent/ListeningExecutorService;IZ)V
  at org.apache.hadoop.fs.s3a.S3AFileSystem.create(S3AFileSystem.java:772)
```

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

No.

### How was this patch tested?

Pass the CI.

Closes #30508 from dongjoon-hyun/SPARK-33212-REVERT.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-02 18:23:48 +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
Thomas Graves f71f34572d [SPARK-33544][SQL] Optimize size of CreateArray/CreateMap to be the size of its children
### What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-32295 added in an optimization to insert a filter for not null and size > 0 when using inner explode/inline. This is fine in most cases but the extra filter is not needed if the explode is with a create array and not using Literals (it already handles LIterals).  When this happens you know that the values aren't null and it has a size.  It already handles the empty array.

The not null check is already optimized out because Createarray and createMap are not nullable, that leaves the size > 0 check. To handle that this PR makes it so that the size > 0 check gets optimized in ConstantFolding to be the size of the children in the array or map.  That makes it a literal and then makes it ultimately be optimized out.

### Why are the changes needed?
remove unneeded filter

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

### How was this patch tested?
Unit tests added and manually tested various cases

Closes #30504 from tgravescs/SPARK-33544.

Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-02 09:50:02 +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
Anton Okolnychyi 478fb7f528 [SPARK-33608][SQL] Handle DELETE/UPDATE/MERGE in PullupCorrelatedPredicates
### What changes were proposed in this pull request?

This PR adds logic to handle DELETE/UPDATE/MERGE plans in `PullupCorrelatedPredicates`.
### Why are the changes needed?

Right now, `PullupCorrelatedPredicates` applies only to filters and unary nodes. As a result, correlated predicates in DELETE/UPDATE/MERGE are not rewritten.

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

No.

### How was this patch tested?

The PR adds 3 new test cases.

Closes #30555 from aokolnychyi/spark-33608.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-01 14:11:01 +00: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
Gabor Somogyi e5bb2937f6 [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API
### What changes were proposed in this pull request?
Deprecated `KafkaConsumer.poll(long)` API calls may cause infinite wait in the driver. In this PR I've added a new `AdminClient` based offset fetching which is turned off by default. There is a new flag named `spark.sql.streaming.kafka.useDeprecatedOffsetFetching` (default: `true`) which can be set to `false` to reach the newly added functionality. The Structured Streaming migration guide contains more information what migration consideration must be done. Please see the following [doc](https://docs.google.com/document/d/1gAh0pKgZUgyqO2Re3sAy-fdYpe_SxpJ6DkeXE8R1P7E/edit?usp=sharing) for further details.

The PR contains the following changes:
* Added `AdminClient` based offset fetching
* GroupId prefix feature removed from driver but only in `AdminClient` based approach (`AdminClient` doesn't need any GroupId)
* GroupId override feature removed from driver but only in `AdminClient` based approach  (`AdminClient` doesn't need any GroupId)
* Additional unit tests
* Code comment changes
* Minor bugfixes here and there
* Removed Kafka auto topic creation feature but only in `AdminClient` based approach (please see doc for rationale). In short, it's super hidden, not sure anybody ever used in production + error prone.
* Added documentation to `ss-migration-guide` and `structured-streaming-kafka-integration`

### Why are the changes needed?
Driver may hang forever.

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

### How was this patch tested?
Existing + additional unit tests.
Cluster test with simple Kafka topic to another topic query.
Documentation:
```
cd docs/
SKIP_API=1 jekyll build
```
Manual webpage check.

Closes #29729 from gaborgsomogyi/SPARK-32032.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-12-01 20:34:00 +09: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
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
xuewei.linxuewei 225c2e2815 [SPARK-33498][SQL][FOLLOW-UP] Deduplicate the unittest by using checkCastWithParseError
### What changes were proposed in this pull request?

Dup code removed in SPARK-33498 as follow-up.

### Why are the changes needed?

Nit.

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

No.

### How was this patch tested?

Existing UT.

Closes #30540 from leanken/leanken-SPARK-33498.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-30 15:36:26 +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
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
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
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
Gengliang Wang d691d85701 [SPARK-33496][SQL] Improve error message of ANSI explicit cast
### What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/30260, there are some type conversions disallowed under ANSI mode.
We should tell users what they can do if they have to use the disallowed casting.

### Why are the changes needed?

Make it more user-friendly.

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

Yes, the error message is improved on casting failure when ANSI mode is enabled
### How was this patch tested?

Unit tests.

Closes #30440 from gengliangwang/improveAnsiCastErrorMSG.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-11-25 23:15:52 +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
Max Gekk 2c5cc36e3f [SPARK-33509][SQL] List partition by names from a V2 table which supports partition management
### What changes were proposed in this pull request?
1. Add new method `listPartitionByNames` to the `SupportsPartitionManagement` interface. It allows to list partitions by partition names and their values.
2. Implement new method in `InMemoryPartitionTable` which is used in DSv2 tests.

### Why are the changes needed?
Currently, the `SupportsPartitionManagement` interface exposes only `listPartitionIdentifiers` which allows to list partitions by partition values. And it requires to specify all values for partition schema fields in the prefix. This restriction does not allow to list partitions by some of partition names (not all of them).

For example, the table `tableA` is partitioned by two column `year` and `month`
```
CREATE TABLE tableA (price int, year int, month int)
USING _
partitioned by (year, month)
```
and has the following partitions:
```
PARTITION(year = 2015, month = 1)
PARTITION(year = 2015, month = 2)
PARTITION(year = 2016, month = 2)
PARTITION(year = 2016, month = 3)
```
If we want to list all partitions with `month = 2`, we have to specify `year` for **listPartitionIdentifiers()** which not always possible as we don't know all `year` values in advance. New method **listPartitionByNames()** allows to specify partition values only for `month`, and get two partitions:
```
PARTITION(year = 2015, month = 2)
PARTITION(year = 2016, month = 2)
```

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

### How was this patch tested?
By running the affected test suite `SupportsPartitionManagementSuite`.

Closes #30452 from MaxGekk/column-names-listPartitionIdentifiers.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-25 12:41:53 +00:00
Gengliang Wang 19f3b89d62 [SPARK-33549][SQL] Remove configuration spark.sql.legacy.allowCastNumericToTimestamp
### What changes were proposed in this pull request?

Remove SQL configuration spark.sql.legacy.allowCastNumericToTimestamp

### Why are the changes needed?

In the current master branch, there is a new configuration `spark.sql.legacy.allowCastNumericToTimestamp` which controls whether to cast Numeric types to Timestamp or not. The default value is true.

After https://github.com/apache/spark/pull/30260, the type conversion between Timestamp type and Numeric type is disallowed in ANSI mode. So, we don't need to a separate configuration `spark.sql.legacy.allowCastNumericToTimestamp` for disallowing the conversion. Users just need to set `spark.sql.ansi.enabled` for the behavior.

As the configuration is not in any released yet, we should remove the configuration to make things simpler.

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

No, since the configuration is not released yet.

### How was this patch tested?

Existing test cases

Closes #30493 from gengliangwang/LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-25 08:59:31 +00: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
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
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
gengjiaan f83fcb1254 [SPARK-33278][SQL][FOLLOWUP] Improve OptimizeWindowFunctions to avoid transfer first to nth_value
### What changes were proposed in this pull request?
https://github.com/apache/spark/pull/30178 provided `OptimizeWindowFunctions` used to transfer `first` to `nth_value`.
If the window frame is `UNBOUNDED PRECEDING AND CURRENT ROW` or `UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING`, `nth_value` has better performance than `first`.
But the `OptimizeWindowFunctions` need to exclude other window frame.

### Why are the changes needed?
 Improve `OptimizeWindowFunctions` to avoid transfer `first` to `nth_value` if the specified window frame isn't `UNBOUNDED PRECEDING AND CURRENT ROW` or `UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING`.

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

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

Closes #30419 from beliefer/SPARK-33278_followup.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-23 14:54:44 +00: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
Terry Kim 60f3a730e4 [SPARK-33515][SQL] Improve exception messages while handling UnresolvedTable
### What changes were proposed in this pull request?

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

Currently, when an identifier is resolved to a view when a table is expected, the following exception message is displayed (e.g., for `COMMENT ON TABLE`):
```
v is a temp view not table.
```
After this PR, the message will be:
```
v is a temp view. 'COMMENT ON TABLE' expects a table.
```

Also, if an identifier is not resolved, the following exception message is currently used:
```
Table not found: t
```
After this PR, the message will be:
```
Table not found for 'COMMENT ON TABLE': 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 #30461 from imback82/unresolved_table_message.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-23 08:54:00 +00:00
Xiao Li c891e025b8 Revert "[SPARK-32481][CORE][SQL] Support truncate table to move data to trash"
### What changes were proposed in this pull request?

This reverts commit 065f17386d, which is not part of any released version. That is, this is an unreleased feature

### Why are the changes needed?

I like the concept of Trash, but I think this PR might just resolve a very specific issue by introducing a mechanism without a proper design doc. This could make the usage more complex.

I think we need to consider the big picture. Trash directory is an important concept. If we decide to introduce it, we should consider all the code paths of Spark SQL that could delete the data, instead of Truncate only. We also need to consider what is the current behavior if the underlying file system does not provide the API `Trash.moveToAppropriateTrash`. Is the exception good? How about the performance when users are using the object store instead of HDFS? Will it impact the GDPR compliance?

In sum, I think we should not merge the PR https://github.com/apache/spark/pull/29552 without the design doc and implementation plan. That is why I reverted it before the code freeze of Spark 3.1

### Does this PR introduce _any_ user-facing change?
Reverted the original commit

### How was this patch tested?
The existing tests.

Closes #30463 from gatorsmile/revertSpark-32481.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-23 17:43:58 +09:00
Liang-Chi Hsieh aa78c05edc [SPARK-33427][SQL][FOLLOWUP] Put key and value into IdentityHashMap sequantially
### What changes were proposed in this pull request?

This follow-up fixes an issue when inserting key/value pairs into `IdentityHashMap` in `SubExprEvaluationRuntime`.

### Why are the changes needed?

The last commits to #30341 follows review comment to use `IdentityHashMap`. Because we leverage `IdentityHashMap` to compare keys in reference, we should not convert expression pairs to Scala map before inserting. Scala map compares keys by equality so we will loss keys with different references.

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

No

### How was this patch tested?

Run benchmark to verify.

Closes #30459 from viirya/SPARK-33427-map.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-23 10:42:28 +09:00
ulysses 6d625ccd5b
[SPARK-33469][SQL] Add current_timezone function
### What changes were proposed in this pull request?

Add a `CurrentTimeZone` function and replace the value at `Optimizer` side.

### Why are the changes needed?

Let user get current timezone easily. Then user can call
```
SELECT current_timezone()
```

Presto: https://prestodb.io/docs/current/functions/datetime.html
SQL Server: https://docs.microsoft.com/en-us/sql/t-sql/functions/current-timezone-transact-sql?view=sql-server-ver15

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

Yes, a new function.

### How was this patch tested?

Add test.

Closes #30400 from ulysses-you/SPARK-33469.

Lead-authored-by: ulysses <youxiduo@weidian.com>
Co-authored-by: ulysses-you <youxiduo@weidian.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-22 15:36:44 -08:00
Max Gekk 530c0a8e28
[SPARK-33505][SQL][TESTS] Fix adding new partitions by INSERT INTO InMemoryPartitionTable
### What changes were proposed in this pull request?
1. Add a hook method to `addPartitionKey()` of `InMemoryTable` which is called per every row.
2. Override `addPartitionKey()` in `InMemoryPartitionTable`, and add partition key every time when new row is inserted to the table.

### Why are the changes needed?
To be able to write unified tests for datasources V1 and V2. Currently, INSERT INTO a V1 table creates partitions but the same doesn't work for the custom catalog `InMemoryPartitionTableCatalog` used in DSv2 tests.

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

### How was this patch tested?
By running the affected test suite `DataSourceV2SQLSuite`.

Closes #30449 from MaxGekk/insert-into-InMemoryPartitionTable.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-20 18:41:25 -08:00
anchovYu de0f50abf4 [SPARK-32670][SQL] Group exception messages in Catalyst Analyzer in one file
### What changes were proposed in this pull request?

Group all messages of `AnalysisExcpetions` created and thrown directly in org.apache.spark.sql.catalyst.analysis.Analyzer in one file.
* Create a new object: `org.apache.spark.sql.CatalystErrors` with many exception-creating functions.
* When the `Analyzer` wants to create and throw a new `AnalysisException`, call functions of `CatalystErrors`

### Why are the changes needed?

This is the sample PR that groups exception messages together in several files. It will largely help with standardization of error messages and its maintenance.

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

No. Error messages remain unchanged.

### How was this patch tested?

No new tests - pass all original tests to make sure it doesn't break any existing behavior.

### Naming of exception functions

All function names ended with `Error`.
* For specific errors like `groupingIDMismatch` and `groupingColInvalid`, directly use them as name, just like `groupingIDMismatchError` and `groupingColInvalidError`.
* For generic errors like `dataTypeMismatch`,
  * if confident with the context, prefix and condition can be added, like `pivotValDataTypeMismatchError`
  * if not sure about the context, add a `For` suffix of the specific component that this exception is related to, like `dataTypeMismatchForDeserializerError`

Closes #29497 from anchovYu/32670.

Lead-authored-by: anchovYu <aureole@sjtu.edu.cn>
Co-authored-by: anchovYu <xyyu15@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-21 08:33:39 +09: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
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 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
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