Commit graph

5010 commits

Author SHA1 Message Date
Terry Kim 0c19497222 [SPARK-33815][SQL] Migrate ALTER TABLE ... SET [SERDE|SERDEPROPERTIES] to use UnresolvedTable to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `ALTER TABLE ... SET [SERDE|SERDEPROPERTIES` 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 `ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]` is not supported for v2 tables.

### Why are the changes needed?

The PR makes the resolution consistent behavior consistent. For example,
```scala
sql("CREATE DATABASE test")
sql("CREATE TABLE spark_catalog.test.t (id bigint, val string) USING csv PARTITIONED BY (id)")
sql("CREATE TEMPORARY VIEW t AS SELECT 2")
sql("USE spark_catalog.test")
sql("ALTER TABLE t SET SERDE 'serdename'") // works fine
```
, but after this PR:
```
sql("ALTER TABLE t SET SERDE 'serdename'")
org.apache.spark.sql.AnalysisException: t is a temp view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES\' expects a table; line 1 pos 0
```
, which is the consistent behavior with other commands.

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

After this PR, `t` in the above example is resolved to a temp view first instead of `spark_catalog.test.t`.

### How was this patch tested?

Updated existing tests.

Closes #30813 from imback82/alter_table_serde_v2.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-17 05:25:51 +00:00
Terry Kim e7e29fd0af [SPARK-33514][SQL][FOLLOW-UP] Remove unused TruncateTableStatement case class
### What changes were proposed in this pull request?

This PR removes unused `TruncateTableStatement`: https://github.com/apache/spark/pull/30457#discussion_r544433820

### Why are the changes needed?

To remove unused `TruncateTableStatement` from #30457.

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

No

### How was this patch tested?

Not needed.

Closes #30811 from imback82/remove_truncate_table_stmt.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-16 14:13:02 -08:00
Kent Yao 728a1298af [SPARK-33806][SQL] limit partition num to 1 when distributing by foldable expressions
### What changes were proposed in this pull request?

It seems a very popular way that people use DISTRIBUTE BY clause with a literal to coalesce partition in the pure SQL data processing.

For example
```
insert into table src select * from values (1), (2), (3) t(a) distribute by 1
```

Users may want the final output to be one single data file, but if the reality is not always true. Spark will always create a file for partition 0 whether it contains data or not, so when the data all goes to a partition(IDX >0), there will be always 2 files there and the part-00000 is empty. On the other hand, a lot of empty tasks will be launched too, this is unnecessary.

When users repeat the insert statement daily, hourly, or minutely, it causes small file issues.

```
spark-sql> set spark.sql.shuffle.partitions=3;drop table if exists test2;create table test2 using parquet as select * from values (1), (2), (3) t(a) distribute by 1;

 kentyaohulk  ~/spark   SPARK-33806  tree /Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20201202/spark-warehouse/test2/ -s
/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20201202/spark-warehouse/test2/
├── [          0]  _SUCCESS
├── [        298]  part-00000-5dc19733-9405-414b-9681-d25c4d3e9ee6-c000.snappy.parquet
└── [        426]  part-00001-5dc19733-9405-414b-9681-d25c4d3e9ee6-c000.snappy.parquet
```

To avoid this, there are some options you can take.

1. use `distribute by null`, let the data go to the partition 0
2. set spark.sql.adaptive.enabled to true for Spark to automatically coalesce
3. using hints instead of `distribute by`
4. set spark.sql.shuffle.partitions to 1

In this PR, we set the partition number to 1 in this particular case.

### Why are the changes needed?

1. avoid small file issues
2. avoid unnecessary empty tasks when no adaptive execution

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

no

### How was this patch tested?

new test

Closes #30800 from yaooqinn/SPARK-33806.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-16 14:09:28 -08:00
Terry Kim 8666d1c39c [SPARK-33800][SQL] Remove command name in AnalysisException message when a relation is not resolved
### What changes were proposed in this pull request?

Based on the discussion https://github.com/apache/spark/pull/30743#discussion_r543124594, this PR proposes to remove the command name in AnalysisException message when a relation is not resolved.

For some of the commands that use `UnresolvedTable`, `UnresolvedView`, and `UnresolvedTableOrView` to resolve an identifier, when the identifier cannot be resolved, the exception will be something like `Table or view not found for 'SHOW TBLPROPERTIES': badtable`. The command name (`SHOW TBLPROPERTIES` in this case) should be dropped to be consistent with other existing commands.

### Why are the changes needed?

To make the exception message consistent.

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

Yes, the exception message will be changed from
```
Table or view not found for 'SHOW TBLPROPERTIES': badtable
```
to
```
Table or view not found: badtable
```
for commands that use `UnresolvedTable`, `UnresolvedView`, and `UnresolvedTableOrView` to resolve an identifier.

### How was this patch tested?

Updated existing tests.

Closes #30794 from imback82/remove_cmd_from_exception_msg.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-16 15:56:50 +00:00
Kent Yao 205d8e40bc [SPARK-32991][SQL] [FOLLOWUP] Reset command relies on session initials first
### What changes were proposed in this pull request?

As a follow-up of https://github.com/apache/spark/pull/30045, we modify the RESET command here to respect the session initial configs per session first then fall back to the `SharedState` conf, which makes each session could maintain a different copy of initial configs for resetting.

### Why are the changes needed?

to make reset command saner.
### Does this PR introduce _any_ user-facing change?

yes, RESET will respect session initials first not always go to the system defaults

### How was this patch tested?

add new tests

Closes #30642 from yaooqinn/SPARK-32991-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-16 14:36:38 +00:00
HyukjinKwon 7845865b8d [SPARK-33803][SQL] Sort table properties by key in DESCRIBE TABLE command
### What changes were proposed in this pull request?

This PR proposes to sort table properties in DESCRIBE TABLE command. This is consistent with DSv2 command as well:
e3058ba17c/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala (L63)

This PR fixes the test case in Scala 2.13 build as well where the table properties have different order in the map.

### Why are the changes needed?

To keep the deterministic and pretty output, and fix the tests in Scala 2.13 build.
See https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-3.2-scala-2.13/49/testReport/junit/org.apache.spark.sql/SQLQueryTestSuite/describe_sql/

```
describe.sql&#010;Expected "...spark_catalog, view.[query.out.col.2=c, view.referredTempFunctionsNames=[], view.catalogAndNamespace.part.1=default]]", but got "...spark_catalog, view.[catalogAndNamespace.part.1=default, view.query.out.col.2=c, view.referredTempFunctionsNames=[]]]" Result did not match for query #29&#010;DESC FORMATTED v
```

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

Yes, it will change the text output from `DESCRIBE [EXTENDED|FORMATTED] table_name`.
Now the table properties are sorted by its key.

### How was this patch tested?

Related unittests were fixed accordingly.

Closes #30799 from HyukjinKwon/SPARK-33803.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-16 13:42:30 +00:00
Terry Kim 62be2483d7 [SPARK-33765][SQL] Migrate UNCACHE TABLE to use UnresolvedRelation to resolve identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `UNCACHE TABLE` to use `UnresolvedRelation` to resolve the table/view identifier in Analyzer as discussed https://github.com/apache/spark/pull/30403/files#r532360022.

### Why are the changes needed?

To resolve the table/view in the analyzer.

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

No

### How was this patch tested?

Updated existing tests

Closes #30743 from imback82/uncache_v2.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-16 05:37:56 +00:00
Anton Okolnychyi 4d56d43838 [SPARK-33735][SQL] Handle UPDATE in ReplaceNullWithFalseInPredicate
### What changes were proposed in this pull request?

This PR adds `UpdateTable` to supported plans in `ReplaceNullWithFalseInPredicate`.

### Why are the changes needed?

This change allows Spark to optimize update conditions like we optimize filters.

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

No.

### How was this patch tested?

This PR extends the existing test cases to also cover `UpdateTable`.

Closes #30787 from aokolnychyi/spark-33735.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-15 13:50:58 -08:00
Wenchen Fan 40c37d69fd [SPARK-33617][SQL][FOLLOWUP] refine the default parallelism SQL config
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/30559 . The default parallelism config in Spark core is not good, as it's unclear where it applies. To not inherit this problem in Spark SQL, this PR refines the default parallelism SQL config, to make it clear that it only applies to leaf nodes.

### Why are the changes needed?

Make the config clearer.

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

It changes an unreleased config.

### How was this patch tested?

existing tests

Closes #30736 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-15 14:16:43 +00:00
gengjiaan 58cb2bae74 [SPARK-33752][SQL] Avoid the getSimpleMessage of AnalysisException adds semicolon repeatedly
### What changes were proposed in this pull request?
The current `getSimpleMessage` of `AnalysisException` may adds semicolon repeatedly. There show an example below:
`select decode()`

The output will be:
```
org.apache.spark.sql.AnalysisException
Invalid number of arguments for function decode. Expected: 2; Found: 0;; line 1 pos 7
```

### Why are the changes needed?
Fix a bug, because it adds semicolon repeatedly.

### Does this PR introduce _any_ user-facing change?
Yes. the message of AnalysisException will be correct.

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

Closes #30724 from beliefer/SPARK-33752.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-15 19:20:01 +09:00
Max Gekk 141e26d65b [SPARK-33767][SQL][TESTS] Unify v1 and v2 ALTER TABLE .. DROP PARTITION tests
### What changes were proposed in this pull request?
1. Move the `ALTER TABLE .. DROP PARTITION` parsing tests to `AlterTableDropPartitionParserSuite`
2. Place v1 tests for `ALTER TABLE .. DROP PARTITION` from `DDLSuite` and v2 tests from `AlterTablePartitionV2SQLSuite` to the common trait `AlterTableDropPartitionSuiteBase`, so, the tests will run for V1, Hive V1 and V2 DS.

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

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

### How was this patch tested?
By running new test suites:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *AlterTableDropPartitionParserSuite"
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
```

Closes #30747 from MaxGekk/unify-alter-table-drop-partition-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-15 05:36:57 +00:00
Terry Kim 366beda54a [SPARK-33785][SQL] Migrate ALTER TABLE ... RECOVER PARTITIONS to use UnresolvedTable to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `ALTER TABLE ... RECOVER PARTITIONS` 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 `ALTER TABLE ... RECOVER PARTITIONS` is not supported for v2 tables.

### Why are the changes needed?

The PR makes the resolution consistent behavior consistent. For example,
```scala
sql("CREATE DATABASE test")
sql("CREATE TABLE spark_catalog.test.t (id bigint, val string) USING csv PARTITIONED BY (id)")
sql("CREATE TEMPORARY VIEW t AS SELECT 2")
sql("USE spark_catalog.test")
sql("ALTER TABLE t RECOVER PARTITIONS") // works fine
```
, but after this PR:
```
sql("ALTER TABLE t RECOVER PARTITIONS")
org.apache.spark.sql.AnalysisException: t is a temp view. 'ALTER TABLE ... RECOVER PARTITIONS' expects a table; line 1 pos 0
```
, which is the consistent behavior with other commands.

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

After this PR, `ALTER TABLE t RECOVER PARTITIONS` in the above example is resolved to a temp view `t` first instead of `spark_catalog.test.t`.

### How was this patch tested?

Updated existing tests.

Closes #30773 from imback82/alter_table_recover_part_v2.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-15 05:23:39 +00:00
Yuming Wang 412d86e711
[SPARK-33771][SQL][TESTS] Fix Invalid value for HourOfAmPm when testing on JDK 14
### What changes were proposed in this pull request?

This pr fix invalid value for HourOfAmPm when testing on JDK 14.

### Why are the changes needed?

Run test on JDK 14.

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

No.

### How was this patch tested?

N/A

Closes #30754 from wangyum/SPARK-33771.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-14 13:34:23 -08:00
Anton Okolnychyi bb60fb1bbd
[SPARK-33779][SQL][FOLLOW-UP] Fix Java Linter error
### What changes were proposed in this pull request?

This PR removes unused imports.

### Why are the changes needed?

These changes are required to fix the build.

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

No.

### How was this patch tested?

Via `dev/lint-java`.

Closes #30767 from aokolnychyi/fix-linter.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-14 11:39:42 -08:00
Anton Okolnychyi 82aca7eb8f [SPARK-33779][SQL] DataSource V2: API to request distribution and ordering on write
### What changes were proposed in this pull request?

This PR adds connector interfaces proposed in the [design doc](https://docs.google.com/document/d/1X0NsQSryvNmXBY9kcvfINeYyKC-AahZarUqg3nS1GQs/edit#) for SPARK-23889.

**Note**: This PR contains a subset of changes discussed in PR #29066.

### Why are the changes needed?

Data sources should be able to request a specific distribution and ordering of data on write. In particular, these scenarios are considered useful:
- global sort
- cluster data and sort within partitions
- local sort within partitions
- no sort

Please see the design doc above for a more detailed explanation of requirements.

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

This PR introduces public changes to the DS V2 by adding a logical write abstraction as we have on the read path as well as additional interfaces to represent distribution and ordering of data (please see the doc for more info).

The existing `Distribution` interface in `read` package is read-specific and not flexible enough like discussed in the design doc. The current proposal is to evolve these interfaces separately until they converge.

### How was this patch tested?

This patch adds only interfaces.

Closes #30706 from aokolnychyi/spark-23889-interfaces.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Ryan Blue <blue@apache.org>
2020-12-14 10:54:18 -08:00
ulysses-you 839d6899ad [SPARK-33733][SQL] PullOutNondeterministic should check and collect deterministic field
### What changes were proposed in this pull request?

The deterministic field is wider than `NonDerterministic`, we should keep same range between pull out and check analysis.

### Why are the changes needed?

For example
```
select * from values(1), (4) as t(c1) order by java_method('java.lang.Math', 'abs', c1)
```

We will get exception since `java_method` deterministic field is false but not a `NonDeterministic`
```
Exception in thread "main" org.apache.spark.sql.AnalysisException: nondeterministic expressions are only allowed in
Project, Filter, Aggregate or Window, found:
 java_method('java.lang.Math', 'abs', t.`c1`) ASC NULLS FIRST
in operator Sort [java_method(java.lang.Math, abs, c1#1) ASC NULLS FIRST], true
               ;;
```

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

Yes.

### How was this patch tested?

Add test.

Closes #30703 from ulysses-you/SPARK-33733.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-14 14:35:24 +00:00
angerszhu 5f9a7fea06 [SPARK-33428][SQL] Conv UDF use BigInt to avoid Long value overflow
### What changes were proposed in this pull request?
Use Long value store  encode value will overflow and return unexpected result, use BigInt to replace Long value and make logical more simple.

### Why are the changes needed?
Fix value  overflow issue

### Does this PR introduce _any_ user-facing change?
People can sue `conf` function to convert value big then LONG.MAX_VALUE

### How was this patch tested?
Added UT

#### BenchMark
```
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.sql.execution.benchmark

import scala.util.Random

import org.apache.spark.benchmark.Benchmark
import org.apache.spark.sql.functions._
object ConvFuncBenchMark extends SqlBasedBenchmark {

  val charset =
    Array[String]("0", "1", "2", "3", "4", "5", "6", "7", "8", "9",
      "A", "B", "C", "D", "E", "F", "G",
      "H", "I", "J", "K", "L", "M", "N",
      "O", "P", "Q", "R", "S", "T",
      "U", "V", "W", "X", "Y", "Z")

  def constructString(from: Int, length: Int): String = {
    val chars = charset.slice(0, from)
    (0 to length).map(x => {
      val v = Random.nextInt(from)
      chars(v)
    }).mkString("")
  }

  private def doBenchmark(cardinality: Long, length: Int, from: Int, toBase: Int): Unit = {
    spark.range(cardinality)
      .withColumn("str", lit(constructString(from, length)))
      .select(conv(col("str"), from, toBase))
      .noop()
  }

  /**
   * Main process of the whole benchmark.
   * Implementations of this method are supposed to use the wrapper method `runBenchmark`
   * for each benchmark scenario.
   */
  override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
    val N = 1000000L
    val benchmark = new Benchmark("conv", N, output = output)
    benchmark.addCase("length 10 from 2 to 16") { _ =>
      doBenchmark(N, 10, 2, 16)
    }

    benchmark.addCase("length 10 from 2 to 10") { _ =>
      doBenchmark(N, 10, 2, 10)
    }

    benchmark.addCase("length 10 from 10 to 16") { _ =>
      doBenchmark(N, 10, 10, 16)
    }

    benchmark.addCase("length 10 from 10 to 36") { _ =>
      doBenchmark(N, 10, 10, 36)
    }

    benchmark.addCase("length 10 from 16 to 10") { _ =>
      doBenchmark(N, 10, 10, 10)
    }

    benchmark.addCase("length 10 from 16 to 36") { _ =>
      doBenchmark(N, 10, 16, 36)
    }

    benchmark.addCase("length 10 from 36 to 10") { _ =>
      doBenchmark(N, 10, 36, 10)
    }

    benchmark.addCase("length 10 from 36 to 16") { _ =>
      doBenchmark(N, 10, 36, 16)
    }

    //
    benchmark.addCase("length 20 from 10 to 16") { _ =>
      doBenchmark(N, 20, 10, 16)
    }

    benchmark.addCase("length 20 from 10 to 36") { _ =>
      doBenchmark(N, 20, 10, 36)
    }

    benchmark.addCase("length 30 from 10 to 16") { _ =>
      doBenchmark(N, 30, 10, 16)
    }

    benchmark.addCase("length 30 from 10 to 36") { _ =>
      doBenchmark(N, 30, 10, 36)
    }

    //
    benchmark.addCase("length 20 from 16 to 10") { _ =>
      doBenchmark(N, 20, 16, 10)
    }

    benchmark.addCase("length 20 from 16 to 36") { _ =>
      doBenchmark(N, 20, 16, 36)
    }

    benchmark.addCase("length 30 from 16 to 10") { _ =>
      doBenchmark(N, 30, 16, 10)
    }

    benchmark.addCase("length 30 from 16 to 36") { _ =>
      doBenchmark(N, 30, 16, 36)
    }

    benchmark.run()
  }

}
```

Result with patch :
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_191-b12 on Mac OS X 10.14.6
Intel(R) Core(TM) i5-8259U CPU  2.30GHz
conv:                                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
length 10 from 2 to 16                               54             73          18         18.7          53.6       1.0X
length 10 from 2 to 10                               43             47           5         23.5          42.5       1.3X
length 10 from 10 to 16                              39             47          12         25.5          39.2       1.4X
length 10 from 10 to 36                              38             42           3         26.5          37.7       1.4X
length 10 from 16 to 10                              39             41           3         25.7          38.9       1.4X
length 10 from 16 to 36                              36             41           4         27.6          36.3       1.5X
length 10 from 36 to 10                              38             40           2         26.3          38.0       1.4X
length 10 from 36 to 16                              37             39           2         26.8          37.2       1.4X
length 20 from 10 to 16                              36             39           2         27.4          36.5       1.5X
length 20 from 10 to 36                              37             39           2         27.2          36.8       1.5X
length 30 from 10 to 16                              37             39           2         27.0          37.0       1.4X
length 30 from 10 to 36                              36             38           2         27.5          36.3       1.5X
length 20 from 16 to 10                              35             38           2         28.3          35.4       1.5X
length 20 from 16 to 36                              34             38           3         29.2          34.3       1.6X
length 30 from 16 to 10                              38             40           2         26.3          38.1       1.4X
length 30 from 16 to 36                              37             38           1         27.2          36.8       1.5X
```
Result without patch:
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_191-b12 on Mac OS X 10.14.6
Intel(R) Core(TM) i5-8259U CPU  2.30GHz
conv:                                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
length 10 from 2 to 16                               66            101          29         15.1          66.1       1.0X
length 10 from 2 to 10                               50             55           5         20.2          49.5       1.3X
length 10 from 10 to 16                              46             51           5         21.8          45.9       1.4X
length 10 from 10 to 36                              43             48           4         23.4          42.7       1.5X
length 10 from 16 to 10                              44             47           4         22.9          43.7       1.5X
length 10 from 16 to 36                              40             44           2         24.7          40.5       1.6X
length 10 from 36 to 10                              40             44           4         25.0          40.1       1.6X
length 10 from 36 to 16                              41             43           2         24.3          41.2       1.6X
length 20 from 10 to 16                              39             41           2         25.7          38.9       1.7X
length 20 from 10 to 36                              40             42           2         24.9          40.2       1.6X
length 30 from 10 to 16                              39             40           1         25.9          38.6       1.7X
length 30 from 10 to 36                              40             41           1         25.0          40.0       1.7X
length 20 from 16 to 10                              40             41           1         25.1          39.8       1.7X
length 20 from 16 to 36                              40             42           2         25.2          39.7       1.7X
length 30 from 16 to 10                              39             42           2         25.6          39.0       1.7X
length 30 from 16 to 36                              39             40           2         25.7          38.8       1.7X
```

Closes #30350 from AngersZhuuuu/SPARK-33428.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-14 14:32:08 +00:00
Terry Kim a84c8d842c [SPARK-33751][SQL] Migrate ALTER VIEW ... AS command to use UnresolvedView to resolve the identifier
### What changes were proposed in this pull request?

This PR migrates `ALTER VIEW ... AS` to use `UnresolvedView` to resolve the 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).

The `TempViewOrV1Table` extractor in `ResolveSessionCatalog.scala` can now be removed as well.

### Why are the changes needed?

To use `UnresolvedView` for view resolution.

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

The exception message changes if a table is found instead of view:
```
// OLD
`tab1` is not a view"
```
```
// NEW
"tab1 is a table. 'ALTER VIEW ... AS' expects a view."
```

### How was this patch tested?

Updated existing tests.

Closes #30723 from imback82/alter_view_as_statement.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-14 08:39:01 +00:00
Linhong Liu b7c8210135 [SPARK-33142][SPARK-33647][SQL][FOLLOW-UP] Add docs and test cases
### What changes were proposed in this pull request?
Addressed comments in PR #30567, including:
1. add test case for SPARK-33647 and SPARK-33142
2. add migration guide
3. add `getRawTempView` and `getRawGlobalTempView` to return the raw view info (i.e. TemporaryViewRelation)
4. other minor code clean

### Why are the changes needed?
Code clean and more test cases

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

### How was this patch tested?
Existing and newly added test cases

Closes #30666 from linhongliu-db/SPARK-33142-followup.

Lead-authored-by: Linhong Liu <linhong.liu@databricks.com>
Co-authored-by: Linhong Liu <67896261+linhongliu-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-14 08:31:50 +00:00
xuewei.linxuewei e7fe92f129 [SPARK-33546][SQL] Enable row format file format validation in CREATE TABLE LIKE
### What changes were proposed in this pull request?

[SPARK-33546] stated the there are three inconsistency behaviors for CREATE TABLE LIKE.

1. CREATE TABLE LIKE does not validate the user-specified hive serde. e.g., STORED AS PARQUET can't be used with ROW FORMAT SERDE.
2. CREATE TABLE LIKE requires STORED AS and ROW FORMAT SERDE to be specified together, which is not necessary.
3. CREATE TABLE LIKE does not respect the default hive serde.

This PR fix No.1, and after investigate, No.2 and No.3 turn out not to be issue.

Within Hive.

CREATE TABLE abc ... ROW FORMAT SERDE 'xxx.xxx.SerdeClass' (Without Stored as) will have
following result. Using the user specific SerdeClass and fetch default input/output format from default textfile format.

```
SerDe Library:          xxx.xxx.SerdeClass
InputFormat:            org.apache.hadoop.mapred.TextInputFormat
OutputFormat:           org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
```

But for
CREATE TABLE dst LIKE src ROW FORMAT SERDE 'xxx.xxx.SerdeClass' (Without Stored as) will just ignore user specific SerdeClass and using (input, output, serdeClass) from src table.

It's better to just throw an exception on such ambiguous behavior, so No.2 is not an issue, but in the PR, we add some comments.

For No.3, in fact, CreateTableLikeCommand is using following logical to try to follow src table's storageFormat if current fileFormat.inputFormat is empty

```
val newStorage = if (fileFormat.inputFormat.isDefined) {
      fileFormat
    } else {
      sourceTableDesc.storage.copy(locationUri = fileFormat.locationUri)
    }
```

If we try to fill the new target table with HiveSerDe.getDefaultStorage if file format and row format is not explicity spefified, it will break the CREATE TABLE LIKE semantic.

### Why are the changes needed?

Bug Fix.

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

No.

### How was this patch tested?

Added UT and Existing UT.

Closes #30705 from leanken/leanken-SPARK-33546.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-14 08:27:18 +00:00
Max Gekk 817f58ddcb [SPARK-33768][SQL] Remove retainData from AlterTableDropPartition
### What changes were proposed in this pull request?
Remove the `retainData` parameter from the logical node `AlterTableDropPartition`.

### Why are the changes needed?
The `AlterTableDropPartition` command reflects the sql statement (see SqlBase.g4):
```
    | ALTER (TABLE | VIEW) multipartIdentifier
        DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE?    #dropTablePartitions
```
but Spark doesn't allow to specify data retention. So, the parameter can be removed to improve code maintenance.

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

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

Closes #30748 from MaxGekk/remove-retainData.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-14 08:16:33 +00:00
Max Gekk 9160d59ae3 [SPARK-33770][SQL][TESTS] Fix the ALTER TABLE .. DROP PARTITION tests that delete files out of partition path
### What changes were proposed in this pull request?
Modify the tests that add partitions with `LOCATION`, and where the number of nested folders in `LOCATION` doesn't match to the number of partitioned columns. In that case, `ALTER TABLE .. DROP PARTITION` tries to access (delete) folder out of the "base" path in `LOCATION`.

The problem belongs to Hive's MetaStore method `drop_partition_common`:
8696c82d07/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (L4876)
which tries to delete empty partition sub-folders recursively starting from the most deeper partition sub-folder up to the base folder. In the case when the number of sub-folder is not equal to the number of partitioned columns `part_vals.size()`, the method will try to list and delete folders out of the base path.

### Why are the changes needed?
To fix test failures like https://github.com/apache/spark/pull/30643#issuecomment-743774733:
```
org.apache.spark.sql.hive.execution.command.AlterTableAddPartitionSuite.ALTER TABLE .. ADD PARTITION Hive V1: SPARK-33521: universal type conversions of partition values
sbt.ForkMain$ForkError: org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: File file:/home/jenkins/workspace/SparkPullRequestBuilder/target/tmp/spark-832cb19c-65fd-41f3-ae0b-937d76c07897 does not exist;
	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:112)
	at org.apache.spark.sql.hive.HiveExternalCatalog.dropPartitions(HiveExternalCatalog.scala:1014)
...
Caused by: sbt.ForkMain$ForkError: org.apache.hadoop.hive.metastore.api.MetaException: File file:/home/jenkins/workspace/SparkPullRequestBuilder/target/tmp/spark-832cb19c-65fd-41f3-ae0b-937d76c07897 does not exist
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.drop_partition_with_environment_context(HiveMetaStore.java:3381)
	at sun.reflect.GeneratedMethodAccessor304.invoke(Unknown Source)
```

The issue can be reproduced by the following steps:
1. Create a base folder, for example: `/Users/maximgekk/tmp/part-location`
2. Create a sub-folder in the base folder and drop permissions for it:
```
$ mkdir /Users/maximgekk/tmp/part-location/aaa
$ chmod a-rwx chmod a-rwx /Users/maximgekk/tmp/part-location/aaa
$ ls -al /Users/maximgekk/tmp/part-location
total 0
drwxr-xr-x   3 maximgekk  staff    96 Dec 13 18:42 .
drwxr-xr-x  33 maximgekk  staff  1056 Dec 13 18:32 ..
d---------   2 maximgekk  staff    64 Dec 13 18:42 aaa
```
3. Create a table with a partition folder in the base folder:
```sql
spark-sql> create table tbl (id int) partitioned by (part0 int, part1 int);
spark-sql> alter table tbl add partition (part0=1,part1=2) location '/Users/maximgekk/tmp/part-location/tbl';
```
4. Try to drop this partition:
```
spark-sql> alter table tbl drop partition (part0=1,part1=2);
20/12/13 18:46:07 ERROR HiveClientImpl:
======================
Attempt to drop the partition specs in table 'tbl' database 'default':
Map(part0 -> 1, part1 -> 2)
In this attempt, the following partitions have been dropped successfully:

The remaining partitions have not been dropped:
[1, 2]
======================

Error in query: org.apache.hadoop.hive.ql.metadata.HiveException: Error accessing file:/Users/maximgekk/tmp/part-location/aaa;
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: Error accessing file:/Users/maximgekk/tmp/part-location/aaa;
```
The command fails because it tries to access to the sub-folder `aaa` that is out of the partition path `/Users/maximgekk/tmp/part-location/tbl`.

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

### How was this patch tested?
By running the affected tests from local IDEA which does not have access to folders out of partition paths.

Closes #30752 from MaxGekk/fix-drop-partition-location.

Lead-authored-by: Max Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-14 15:56:46 +09:00
Gengliang Wang 6e862792fb [SPARK-33723][SQL] ANSI mode: Casting String to Date should throw exception on parse error
### What changes were proposed in this pull request?

Currently, when casting a string as timestamp type in ANSI mode, Spark throws a runtime exception on parsing error.
However, the result for casting a string to date is always null. We should throw an exception on parsing error as well.

### Why are the changes needed?

Add missing feature for ANSI mode

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

Yes for ANSI mode, Casting string to date will throw an exception on parsing error

### How was this patch tested?

Unit test

Closes #30687 from gengliangwang/castDate.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-14 10:22:37 +09:00
Liang-Chi Hsieh 45af3c9688
[SPARK-33764][SS] Make state store maintenance interval as SQL config
### What changes were proposed in this pull request?

Currently the maintenance interval is hard-coded in `StateStore`. This patch proposes to make it as SQL config.

### Why are the changes needed?

Currently the maintenance interval is hard-coded in `StateStore`. For consistency reason, it should be placed together with other SS configs together. SQLConf also has a better way to have doc and default value setting.

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

Yes. Previously users use Spark config to set the maintenance interval. Now they could use SQL config to set it.

### How was this patch tested?

Unit test.

Closes #30741 from viirya/maintenance-interval-sqlconfig.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-13 14:57:09 -08:00
ulysses-you 5bab27e00b [SPARK-33526][SQL] Add config to control if cancel invoke interrupt task on thriftserver
### What changes were proposed in this pull request?

This PR add a new config `spark.sql.thriftServer.forceCancel` to give user a way to interrupt task when cancel statement.

### Why are the changes needed?

After [#29933](https://github.com/apache/spark/pull/29933), we support cancel query if timeout, but the default behavior of `SparkContext.cancelJobGroups` won't interrupt task and just let task finish by itself. In some case it's dangerous, e.g., data skew or exists a heavily shuffle. A task will hold in a long time after do cancel and the resource will not release.

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

Yes, a new config.

### How was this patch tested?

Add test.

Closes #30481 from ulysses-you/SPARK-33526.

Lead-authored-by: ulysses-you <ulyssesyou18@gmail.com>
Co-authored-by: ulysses-you <youxiduo@weidian.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-12 00:52:33 +09:00
Max Gekk 8b97b19ffa [SPARK-33706][SQL] Require fully specified partition identifier in partitionExists()
### What changes were proposed in this pull request?
1. Check that the partition identifier passed to `SupportsPartitionManagement.partitionExists()` is fully specified (specifies all values of partition fields).
2. Remove the custom implementation of `partitionExists()` from `InMemoryPartitionTable`, and re-use the default implementation from `SupportsPartitionManagement`.

### Why are the changes needed?
The method is supposed to check existence of one partition but currently it can return `true` for partially specified partition. This can lead to incorrect commands behavior, for instance the commands could modify or place data in the middle of partition path.

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

### How was this patch tested?
By running existing test suites:
```
$ build/sbt "test:testOnly *AlterTablePartitionV2SQLSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *SupportsPartitionManagementSuite"
```

Closes #30667 from MaxGekk/check-len-partitionExists.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-11 12:48:40 +00:00
Terry Kim 8f5db716fa [SPARK-33654][SQL] Migrate CACHE TABLE to use UnresolvedRelation to resolve identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `CACHE TABLE` to use `UnresolvedRelation` to resolve the table/view identifier in Analyzer as discussed https://github.com/apache/spark/pull/30403/files#r532360022.

### Why are the changes needed?

To resolve the table in the analyzer.

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

No

### How was this patch tested?

Existing tests

Closes #30598 from imback82/cache_v2.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-11 12:39:58 +00:00
Kousuke Saruta 8377aca60a [SPARK-33527][SQL][FOLLOWUP] Fix the scala 2.13 build failure
### What changes were proposed in this pull request?

This PR fixes the Scala 2.13 build failure brought by #30479 .

### Why are the changes needed?

To pass Scala 2.13 build.

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

No.

### How was this patch tested?

Should be done byGitHub Actions.

Closes #30727 from sarutak/fix-scala213-build-failure.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-11 01:53:41 -08:00
Josh Soref c05f6f98b6 [MINOR][SQL] Spelling: enabled - legacy_setops_precedence_enbled
### What changes were proposed in this pull request?

Replace `legacy_setops_precedence_enbled` with `legacy_setops_precedence_enabled`

Alternatively, `legacy_setops_precedence_enabled` could be added, and `legacy_setops_precedence_enbled` retained, and if set the code could honor it and warn about the deprecated spelling.

### Why are the changes needed?

`enabled` is misspelled in `legacy_setops_precedence_enbled`

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

Yes.

It would break current consumers.
Examples include:
* https://www.programmersought.com/article/87752082924/
* 125d873c38/fugue_sql/_antlr/fugue_sqlLexer.py
* https://github.com/search?q=legacy_setops_precedence_enbled&type=code

### How was this patch tested?

It's been included in #30323 for a while (and is now split out here)

Closes #30677 from jsoref/spelling-enabled.

Authored-by: Josh Soref <jsoref@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-11 06:49:45 +00:00
gengjiaan 24d7e45d31 [SPARK-33527][SQL] Extend the function of decode so as consistent with mainstream databases
### What changes were proposed in this pull request?
In Spark, decode(bin, charset) - Decodes the first argument using the second argument character set.

Unfortunately this is NOT what any other SQL vendor understands `DECODE` to do.
`DECODE` generally is a short hand for a simple case expression:

```
SELECT DECODE(c1, 1, 'Hello', 2, 'World', '!') FROM (VALUES (1), (2), (3)) AS T(c1)
=>
(Hello),
(World)
(!)
```
There are some mainstream database support the syntax.
**Oracle**
https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/DECODE.html#GUID-39341D91-3442-4730-BD34-D3CF5D4701CE
**Vertica**
https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/Functions/String/DECODE.htm?tocpath=SQL%20Reference%20Manual%7CSQL%20Functions%7CString%20Functions%7C_____10
**DB2**
https://www.ibm.com/support/knowledgecenter/SSGU8G_14.1.0/com.ibm.sqls.doc/ids_sqs_1447.htm
**Redshift**
https://docs.aws.amazon.com/redshift/latest/dg/r_DECODE_expression.html
**Pig**
https://pig.apache.org/docs/latest/api/org/apache/pig/piggybank/evaluation/decode/Decode.html
**Teradata**
https://docs.teradata.com/reader/756LNiPSFdY~4JcCCcR5Cw/jtCpCycpEaXESG4d63kMjg
**Snowflake**
https://docs.snowflake.com/en/sql-reference/functions/decode.html

### Why are the changes needed?
It is very useful.

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

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

Closes #30479 from beliefer/SPARK-33527.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-11 05:52:33 +00:00
Linhong Liu 1554977670 [SPARK-33692][SQL] View should use captured catalog and namespace to lookup function
### What changes were proposed in this pull request?
Using the view captured catalog and namespace to lookup function, so the view
referred functions won't be overridden by newly created function with the same name,
but different database or function type (i.e. temporary function)

### Why are the changes needed?
bug fix, without this PR, changing database or create a temporary function with
the same name may cause failure when querying a view.

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

### How was this patch tested?
newly added and existing test cases.

Closes #30662 from linhongliu-db/SPARK-33692.

Lead-authored-by: Linhong Liu <linhong.liu@databricks.com>
Co-authored-by: Linhong Liu <67896261+linhongliu-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-10 09:14:07 +00:00
gengjiaan cef28c2c51 [SPARK-32670][SQL][FOLLOWUP] Group exception messages in Catalyst Analyzer in one file
### What changes were proposed in this pull request?
This PR follows up https://github.com/apache/spark/pull/29497.
Because https://github.com/apache/spark/pull/29497 just give us an example to group all `AnalysisExcpetion` in Analyzer into QueryCompilationErrors.
This PR group other `AnalysisExcpetion` into QueryCompilationErrors.

### Why are the changes needed?
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.

Closes #30564 from beliefer/SPARK-32670-followup.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-10 08:38:24 +00:00
Terry Kim b112e2bfa6 [SPARK-33714][SQL] Migrate ALTER VIEW ... SET/UNSET TBLPROPERTIES commands to use UnresolvedView to resolve the identifier
### What changes were proposed in this pull request?

This PR adds `allowTemp` flag to `UnresolvedView` so that `Analyzer` can check whether to resolve temp views or not.

This PR also migrates `ALTER VIEW ... SET/UNSET TBLPROPERTIES` to use `UnresolvedView` 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).

### Why are the changes needed?

To use `UnresolvedView` for view resolution.

One benefit is that the exception message is better for `ALTER VIEW ... SET/UNSET TBLPROPERTIES`. Before, if a temp view is passed, you will just get `NoSuchTableException` with `Table or view 'tmpView' not found in database 'default'`. But with this PR, you will get more description exception message: `tmpView is a temp view. ALTER VIEW ... SET TBLPROPERTIES expects a permanent view`.

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

The exception message changes as describe above.

### How was this patch tested?

Updated existing tests.

Closes #30676 from imback82/alter_view_set_unset_properties.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-10 05:18:34 +00:00
Max Gekk af37c7f411 [SPARK-33558][SQL][TESTS] Unify v1 and v2 ALTER TABLE .. ADD PARTITION tests
### What changes were proposed in this pull request?
1. Move the `ALTER TABLE .. ADD PARTITION` parsing tests to `AlterTableAddPartitionParserSuite`
2. Place v1 tests for `ALTER TABLE .. ADD PARTITION` from `DDLSuite` and v2 tests from `AlterTablePartitionV2SQLSuite` to the common trait `AlterTableAddPartitionSuiteBase`, so, the tests will run for V1, Hive V1 and V2 DS.

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

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

### How was this patch tested?
By running new test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableAddPartitionSuite"
```

Closes #30685 from MaxGekk/unify-alter-table-add-partition-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-10 04:54:52 +00:00
Anton Okolnychyi fa9ce1d4e8
[SPARK-33722][SQL] Handle DELETE in ReplaceNullWithFalseInPredicate
### What changes were proposed in this pull request?

This PR adds `DeleteFromTable` to supported plans in `ReplaceNullWithFalseInPredicate`.

### Why are the changes needed?

This change allows Spark to optimize delete conditions like we optimize filters.

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

No.

### How was this patch tested?

This PR extends the existing test cases to also cover `DeleteFromTable`.

Closes #30688 from aokolnychyi/spark-33722.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-09 11:42:54 -08:00
HyukjinKwon b5399d4ef1 [SPARK-33071][SPARK-33536][SQL][FOLLOW-UP] Rename deniedMetadataKeys to nonInheritableMetadataKeys in Alias
### What changes were proposed in this pull request?

This PR is a followup of https://github.com/apache/spark/pull/30488. This PR proposes to rename `Alias.deniedMetadataKeys` to `Alias.nonInheritableMetadataKeys` to make it less confusing.

### Why are the changes needed?

To make it easier to maintain and read.

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

No. This is rather a code cleanup.

### How was this patch tested?

Ran the unittests written in the previous PR manually. Jenkins and GitHub Actions in this PR should also test them.

Closes #30682 from HyukjinKwon/SPARK-33071-SPARK-33536.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-09 20:26:18 +09:00
Terry Kim 29fed23ba1 [SPARK-33703][SQL] Migrate MSCK REPAIR TABLE to use UnresolvedTable to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `MSCK REPAIR 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 `MSCK REPAIR TABLE` is not supported for v2 tables.

### Why are the changes needed?

The PR makes the resolution consistent behavior consistent. For example,
```scala
sql("CREATE DATABASE test")
sql("CREATE TABLE spark_catalog.test.t (id bigint, val string) USING csv PARTITIONED BY (id)")
sql("CREATE TEMPORARY VIEW t AS SELECT 2")
sql("USE spark_catalog.test")
sql("MSCK REPAIR TABLE t") // works fine
```
, but after this PR:
```
sql("MSCK REPAIR TABLE t")
org.apache.spark.sql.AnalysisException: t is a temp view. 'MSCK REPAIR TABLE' expects a table; line 1 pos 0
```
, which is the consistent behavior with other commands.

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

After this PR, `MSCK REPAIR TABLE t` in the above example is resolved to a temp view `t` first instead of `spark_catalog.test.t`.

### How was this patch tested?

Updated existing tests.

Closes #30664 from imback82/repair_table_V2.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-09 05:06:37 +00:00
Wenchen Fan 6fd234503c
[SPARK-32110][SQL] normalize special floating numbers in HyperLogLog++
### What changes were proposed in this pull request?

Currently, Spark treats 0.0 and -0.0 semantically equal, while it still retains the difference between them so that users can see -0.0 when displaying the data set.

The comparison expressions in Spark take care of the special floating numbers and implement the correct semantic. However, Spark doesn't always use these comparison expressions to compare values, and we need to normalize the special floating numbers before comparing them in these places:
1. GROUP BY
2. join keys
3. window partition keys

This PR fixes one more place that compares values without using comparison expressions: HyperLogLog++

### Why are the changes needed?

Fix the query result

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

Yes, the result of HyperLogLog++ becomes correct now.

### How was this patch tested?

a new test case, and a few more test cases that pass before this PR to improve test coverage.

Closes #30673 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-08 11:41:35 -08:00
Terry Kim c05ee06f5b [SPARK-33685][SQL] Migrate DROP VIEW command to use UnresolvedView to resolve the identifier
### What changes were proposed in this pull request?

This PR introduces `UnresolvedView` in the resolution framework to resolve the identifier.

This PR then migrates `DROP VIEW` to use `UnresolvedView` 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).

### Why are the changes needed?

To use `UnresolvedView` for view resolution. Note that 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 #30636 from imback82/drop_view_v2.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-08 14:07:58 +00:00
Max Gekk 2b30dde249 [SPARK-33688][SQL] Migrate SHOW TABLE EXTENDED to new resolution framework
### What changes were proposed in this pull request?
1. Remove old statement `ShowTableStatement`
2. Introduce new command `ShowTableExtended` for  `SHOW TABLE EXTENDED`.

This PR is the first step of new V2 implementation of `SHOW TABLE EXTENDED`, see SPARK-33393.

### Why are the changes needed?
This is a part of effort to make the relation lookup behavior consistent: SPARK-29900.

### Does this PR introduce _any_ user-facing change?
The changes should not affect V1 tables. For V2, Spark outputs the error:
```
SHOW TABLE EXTENDED is not supported for v2 tables.
```

### How was this patch tested?
By running `SHOW TABLE EXTENDED` tests:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowTablesSuite"
```

Closes #30645 from MaxGekk/show-table-extended-statement.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-08 12:08:22 +00:00
luluorta 99613cd581 [SPARK-33677][SQL] Skip LikeSimplification rule if pattern contains any escapeChar
### What changes were proposed in this pull request?
`LikeSimplification` rule does not work correctly for many cases that have patterns containing escape characters, for example:

`SELECT s LIKE 'm%aca' ESCAPE '%' FROM t`
`SELECT s LIKE 'maacaa' ESCAPE 'a' FROM t`

For simpilicy, this PR makes this rule just be skipped if `pattern` contains any `escapeChar`.

### Why are the changes needed?
Result corrupt.

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

### How was this patch tested?
Added Unit test.

Closes #30625 from luluorta/SPARK-33677.

Authored-by: luluorta <luluorta@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-12-08 20:45:25 +09:00
Dongjoon Hyun 031c5ef280
[SPARK-33679][SQL] Enable spark.sql.adaptive.enabled by default
### What changes were proposed in this pull request?

This PR aims to enable `spark.sql.adaptive.enabled` by default for Apache Spark **3.2.0**.

### Why are the changes needed?

By switching the default for Apache Spark 3.2, the whole community can focus more on the stabilizing this feature in the various situation more seriously.

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

Yes, but this is an improvement and it's supposed to have no bugs.

### How was this patch tested?

Pass the CIs.

Closes #30628 from dongjoon-hyun/SPARK-33679.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-07 23:10:35 -08:00
Terry Kim 5aefc49b0f [SPARK-33664][SQL] Migrate ALTER TABLE ... RENAME TO to use UnresolvedTableOrView to resolve identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `ALTER [TABLE|ViEW] ... RENAME TO` 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).

### Why are the changes needed?

To use `UnresolvedTableOrView` for table/view resolution. Note that `AlterTableRenameCommand` 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 #30610 from imback82/rename_v2.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-08 03:54:16 +00:00
Wenchen Fan c0874ba9f1
[SPARK-33480][SQL][FOLLOWUP] do not expose user data in error message
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/30412. This PR updates the error message of char/varchar table insertion length check, to not expose user data.

### Why are the changes needed?

This is risky to expose user data in the error message, especially the string data, as it may contain sensitive data.

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

no

### How was this patch tested?

updated tests

Closes #30653 from cloud-fan/minor2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-07 13:35:37 -08:00
Wenchen Fan 6aff215077 [SPARK-33693][SQL] deprecate spark.sql.hive.convertCTAS
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/30554 . Now we have a new config for converting CREATE TABLE, we don't need the old config that only works for CTAS.

### Why are the changes needed?

It's confusing for having two config while one can cover another completely.

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

no, it's deprecating not removing.

### How was this patch tested?

N/A

Closes #30651 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-07 10:50:31 -08:00
Josh Soref c62b84a043 [MINOR] Spelling sql not core
### What changes were proposed in this pull request?

This PR intends to fix typos in the sub-modules:
* `sql/catalyst`
* `sql/hive-thriftserver`
* `sql/hive`

Split per srowen https://github.com/apache/spark/pull/30323#issuecomment-728981618

NOTE: The misspellings have been reported at 706a726f87 (commitcomment-44064356)

### Why are the changes needed?

Misspelled words make it harder to read / understand content.

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

There are various fixes to documentation, etc...

### How was this patch tested?

No testing was performed

Closes #30532 from jsoref/spelling-sql-not-core.

Authored-by: Josh Soref <jsoref@users.noreply.github.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-12-07 08:40:29 -06:00
Kent Yao da72b87374 [SPARK-33641][SQL] Invalidate new char/varchar types in public APIs that produce incorrect results
### What changes were proposed in this pull request?

In this PR, we suppose to narrow the use cases of the char/varchar data types, of which are invalid now or later

### Why are the changes needed?
1. udf
```scala
scala> spark.udf.register("abcd", () => "12345", org.apache.spark.sql.types.VarcharType(2))

scala> spark.sql("select abcd()").show
scala.MatchError: CharType(2) (of class org.apache.spark.sql.types.VarcharType)
  at org.apache.spark.sql.catalyst.encoders.RowEncoder$.externalDataTypeFor(RowEncoder.scala:215)
  at org.apache.spark.sql.catalyst.encoders.RowEncoder$.externalDataTypeForInput(RowEncoder.scala:212)
  at org.apache.spark.sql.catalyst.expressions.objects.ValidateExternalType.<init>(objects.scala:1741)
  at org.apache.spark.sql.catalyst.encoders.RowEncoder$.$anonfun$serializerFor$3(RowEncoder.scala:175)
  at scala.collection.TraversableLike.$anonfun$flatMap$1(TraversableLike.scala:245)
  at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
  at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
  at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:198)
  at scala.collection.TraversableLike.flatMap(TraversableLike.scala:245)
  at scala.collection.TraversableLike.flatMap$(TraversableLike.scala:242)
  at scala.collection.mutable.ArrayOps$ofRef.flatMap(ArrayOps.scala:198)
  at org.apache.spark.sql.catalyst.encoders.RowEncoder$.serializerFor(RowEncoder.scala:171)
  at org.apache.spark.sql.catalyst.encoders.RowEncoder$.apply(RowEncoder.scala:66)
  at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:99)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:768)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:96)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:611)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:768)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:606)
  ... 47 elided
```

2. spark.createDataframe

```
scala> spark.createDataFrame(spark.read.text("README.md").rdd, new org.apache.spark.sql.types.StructType().add("c", "char(1)")).show
+--------------------+
|                   c|
+--------------------+
|      # Apache Spark|
|                    |
|Spark is a unifie...|
|high-level APIs i...|
|supports general ...|
|rich set of highe...|
|MLlib for machine...|
|and Structured St...|
|                    |
|<https://spark.ap...|
|                    |
|[![Jenkins Build]...|
|[![AppVeyor Build...|
|[![PySpark Covera...|
|                    |
|                    |
```

3. reader.schema

```
scala> spark.read.schema("a varchar(2)").text("./README.md").show(100)
+--------------------+
|                   a|
+--------------------+
|      # Apache Spark|
|                    |
|Spark is a unifie...|
|high-level APIs i...|
|supports general ...|
```
4. etc

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

NO, we intend to avoid protentical breaking change

### How was this patch tested?

new tests

Closes #30586 from yaooqinn/SPARK-33641.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-07 13:40:15 +00:00
Yuming Wang 1e0c006748 [SPARK-33617][SQL] Add default parallelism configuration for Spark SQL queries
### What changes were proposed in this pull request?

This pr add default parallelism configuration(`spark.sql.default.parallelism`) for Spark SQL and make it effective for `LocalTableScan`.

### Why are the changes needed?

Avoid generating small files for INSERT INTO TABLE from VALUES, for example:
```sql
CREATE TABLE t1(id int) USING parquet;
INSERT INTO TABLE t1 VALUES (1), (2), (3), (4), (5), (6), (7), (8);
```

Before this pr:
```
-rw-r--r-- 1 root root 421 Dec  1 01:54 part-00000-4d5a3a89-2995-4328-b2ae-908febbbaf4a-c000.snappy.parquet
-rw-r--r-- 1 root root 421 Dec  1 01:54 part-00001-4d5a3a89-2995-4328-b2ae-908febbbaf4a-c000.snappy.parquet
-rw-r--r-- 1 root root 421 Dec  1 01:54 part-00002-4d5a3a89-2995-4328-b2ae-908febbbaf4a-c000.snappy.parquet
-rw-r--r-- 1 root root 421 Dec  1 01:54 part-00003-4d5a3a89-2995-4328-b2ae-908febbbaf4a-c000.snappy.parquet
-rw-r--r-- 1 root root 421 Dec  1 01:54 part-00004-4d5a3a89-2995-4328-b2ae-908febbbaf4a-c000.snappy.parquet
-rw-r--r-- 1 root root 421 Dec  1 01:54 part-00005-4d5a3a89-2995-4328-b2ae-908febbbaf4a-c000.snappy.parquet
-rw-r--r-- 1 root root 421 Dec  1 01:54 part-00006-4d5a3a89-2995-4328-b2ae-908febbbaf4a-c000.snappy.parquet
-rw-r--r-- 1 root root 421 Dec  1 01:54 part-00007-4d5a3a89-2995-4328-b2ae-908febbbaf4a-c000.snappy.parquet
-rw-r--r-- 1 root root   0 Dec  1 01:54 _SUCCESS
```

After this pr and set `spark.sql.files.minPartitionNum` to 1:
```
-rw-r--r-- 1 root root 452 Dec  1 01:59 part-00000-6de50c79-e305-4f8d-b6ae-39f46b2619c6-c000.snappy.parquet
-rw-r--r-- 1 root root   0 Dec  1 01:59 _SUCCESS
```

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

No.

### How was this patch tested?

Unit test.

Closes #30559 from wangyum/SPARK-33617.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <yumwang@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-07 21:36:52 +09:00
Max Gekk 26c0493318 [SPARK-33676][SQL] Require exact matching of partition spec to the schema in V2 ALTER TABLE .. ADD/DROP PARTITION
### What changes were proposed in this pull request?
Check that partitions specs passed to v2 `ALTER TABLE .. ADD/DROP PARTITION` exactly match to the partition schema (all partition fields from the schema are specified in partition specs).

### Why are the changes needed?
1. To have the same behavior as V1 `ALTER TABLE .. ADD/DROP PARTITION` that output the error:
```sql
spark-sql> create table tab1 (id int, a int, b int) using parquet partitioned by (a, b);
spark-sql> ALTER TABLE tab1 ADD PARTITION (A='9');
Error in query: Partition spec is invalid. The spec (a) must match the partition spec (a, b) defined in table '`default`.`tab1`';
```
2. To prevent future errors caused by not fully specified partition specs.

### Does this PR introduce _any_ user-facing change?
Yes. The V2 implementation of `ALTER TABLE .. ADD/DROP PARTITION` output the same error as V1 commands.

### How was this patch tested?
By running the test suite with new UT:
```
$ build/sbt "test:testOnly *AlterTablePartitionV2SQLSuite"
```

Closes #30624 from MaxGekk/add-partition-full-spec.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-07 08:14:36 +00:00
Chao Sun e857e06452
[SPARK-33652][SQL] DSv2: DeleteFrom should refresh cache
### What changes were proposed in this pull request?

This changes `DeleteFromTableExec` to also refresh caches referencing the original table, by passing the `refreshCache` callback to the class. Note that in order to construct the callback, I have to change `DataSourceV2ScanRelation` to contain a `DataSourceV2Relation` instead of a `Table`.

### Why are the changes needed?

Currently DSv2 delete from table doesn't refresh caches. This could lead to correctness issue if the staled cache is queried later.

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

Yes. Now delete from table in v2 also refreshes cache.

### How was this patch tested?

Added a test case.

Closes #30597 from sunchao/SPARK-33652.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-06 01:14:22 -08:00
Wenchen Fan 1b4e35d1a8
[SPARK-33651][SQL] Allow CREATE EXTERNAL TABLE with LOCATION for data source tables
### What changes were proposed in this pull request?

This PR removes the restriction and allows CREATE EXTERNAL TABLE with LOCATION for data source tables. It also moves the check from the analyzer rule `ResolveSessionCatalog` to `SessionCatalog`, so that v2 session catalog can overwrite it.

### Why are the changes needed?

It's an unnecessary behavior difference that Hive serde table can be created with `CREATE EXTERNAL TABLE` if LOCATION is present, while data source table doesn't allow `CREATE EXTERNAL TABLE` at all.

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

Yes, now `CREATE EXTERNAL TABLE ... USING ... LOCATION ...` is allowed.

### How was this patch tested?

new tests

Closes #30595 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-04 16:48:31 -08:00
Dongjoon Hyun de9818f043
[SPARK-33662][BUILD] Setting version to 3.2.0-SNAPSHOT
### What changes were proposed in this pull request?

This PR aims to update `master` branch version to 3.2.0-SNAPSHOT.

### Why are the changes needed?

Start to prepare Apache Spark 3.2.0.

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

N/A.

### How was this patch tested?

Pass the CIs.

Closes #30606 from dongjoon-hyun/SPARK-3.2.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-04 14:10:42 -08:00
Wenchen Fan acc211d2cf [SPARK-33141][SQL][FOLLOW-UP] Store the max nested view depth in AnalysisContext
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/30289. It removes the hack in `View.effectiveSQLConf`, by putting the max nested view depth in `AnalysisContext`. Then we don't get the max nested view depth from the active SQLConf, which keeps changing during nested view resolution.

### Why are the changes needed?

remove hacks.

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

No

### How was this patch tested?

If I just remove the hack, `SimpleSQLViewSuite.restrict the nested level of a view` fails. With this fix, it passes again.

Closes #30575 from cloud-fan/view.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-04 14:01:15 +00:00
Jungtaek Lim (HeartSaVioR) 233a8494c8 [SPARK-27237][SS] Introduce State schema validation among query restart
## What changes were proposed in this pull request?

Please refer the description of [SPARK-27237](https://issues.apache.org/jira/browse/SPARK-27237) to see rationalization of this patch.

This patch proposes to introduce state schema validation, via storing key schema and value schema to `schema` file (for the first time) and verify new key schema and value schema for state are compatible with existing one. To be clear for definition of "compatible", state schema is "compatible" when number of fields are same and data type for each field is same - Spark has been allowing rename of field.

This patch will prevent query run which has incompatible state schema, which would reduce the chance to get indeterministic behavior (actually renaming of field is also the smell of semantically incompatible, but end users could just modify its name so we can't say) as well as providing more informative error message.

## How was this patch tested?

Added UTs.

Closes #24173 from HeartSaVioR/SPARK-27237.

Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-04 19:33:11 +09:00
Max Gekk 94c144bdd0 [SPARK-33571][SQL][DOCS] Add a ref to INT96 config from the doc for spark.sql.legacy.parquet.datetimeRebaseModeInWrite/Read
### What changes were proposed in this pull request?
For the SQL configs `spark.sql.legacy.parquet.datetimeRebaseModeInWrite` and `spark.sql.legacy.parquet.datetimeRebaseModeInRead`, improve their descriptions by:
1. Explicitly document on which parquet types, those configs influence on
2. Refer to corresponding configs for `INT96`

### Why are the changes needed?
To avoid user confusions like reposted in SPARK-33571, and make the config description more precise.

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

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

Closes #30596 from MaxGekk/clarify-rebase-docs.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-04 16:26:07 +09:00
Gengliang Wang e8380665c7 [SPARK-33658][SQL] Suggest using Datetime conversion functions for invalid ANSI casting
### What changes were proposed in this pull request?

Suggest users using Datetime conversion functions in the error message of invalid ANSI explicit casting.

### Why are the changes needed?

In ANSI mode, explicit cast between DateTime types and Numeric types is not allowed.
As of now, we have introduced new functions `UNIX_SECONDS`/`UNIX_MILLIS`/`UNIX_MICROS`/`UNIX_DATE`/`DATE_FROM_UNIX_DATE`, we can show suggestions to users so that they can complete these type conversions precisely and easily in ANSI mode.

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

Yes, better error messages

### How was this patch tested?

Unit test

Closes #30603 from gengliangwang/improveErrorMsgOfExplicitCast.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-04 16:24:41 +09:00
Linhong Liu e02324f2dd [SPARK-33142][SPARK-33647][SQL] Store SQL text for SQL temp view
### What changes were proposed in this pull request?
Currently, in spark, the temp view is saved as its analyzed logical plan, while the permanent view
is kept in HMS with its origin SQL text. As a result, permanent and temporary views have
different behaviors in some cases. In this PR we store the SQL text for temporary view in order
to unify the behavior between permanent and temporary views.

### Why are the changes needed?
to unify the behavior between permanent and temporary views

### Does this PR introduce _any_ user-facing change?
Yes, with this PR, the temporary view will be re-analyzed when it's referred. So if the
underlying datasource changed, the view will also be updated.

### How was this patch tested?
existing and newly added test cases

Closes #30567 from linhongliu-db/SPARK-33142.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-04 06:48:49 +00:00
Gengliang Wang 29e415deac [SPARK-33649][SQL][DOC] Improve the doc of spark.sql.ansi.enabled
### What changes were proposed in this pull request?

Improve the documentation of SQL configuration `spark.sql.ansi.enabled`

### Why are the changes needed?

As there are more and more new features under the SQL configuration `spark.sql.ansi.enabled`, we should make it more clear about:
1. what exactly it is
2. where can users find all the features of the ANSI mode
3. whether all the features are exactly from the SQL standard

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

No

### How was this patch tested?

It's just doc change.

Closes #30593 from gengliangwang/reviseAnsiDoc.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-12-04 10:58:41 +08:00
Max Gekk 85949588b7 [SPARK-33650][SQL] Fix the error from ALTER TABLE .. ADD/DROP PARTITION for non-supported partition management table
### What changes were proposed in this pull request?
In the PR, I propose to change the order of post-analysis checks for the `ALTER TABLE .. ADD/DROP PARTITION` command, and perform the general check (does the table support partition management at all) before specific checks.

### Why are the changes needed?
The error message for the table which doesn't support partition management can mislead users:
```java
PartitionSpecs are not resolved;;
'AlterTableAddPartition [UnresolvedPartitionSpec(Map(id -> 1),None)], false
+- ResolvedTable org.apache.spark.sql.connector.InMemoryTableCatalog2fd64b11, ns1.ns2.tbl, org.apache.spark.sql.connector.InMemoryTable5d3ff859
```
because it says nothing about the root cause of the issue.

### Does this PR introduce _any_ user-facing change?
Yes. After the change, the error message will be:
```
Table ns1.ns2.tbl can not alter partitions
```

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

Closes #30594 from MaxGekk/check-order-AlterTablePartition.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-03 16:43:15 -08:00
Anton Okolnychyi aa13e207c9
[SPARK-33623][SQL] Add canDeleteWhere to SupportsDelete
### What changes were proposed in this pull request?

This PR provides us with a way to check if a data source is going to reject the delete via `deleteWhere` at planning time.

### Why are the changes needed?

The only way to support delete statements right now is to implement ``SupportsDelete``. According to its Javadoc, that interface is meant for cases when we can delete data without much effort (e.g. like deleting a complete partition in a Hive table).

This PR actually provides us with a way to check if a data source is going to reject the delete via `deleteWhere` at planning time instead of just getting an exception during execution. In the future, we can use this functionality to decide whether Spark should rewrite this delete and execute a distributed query or it can just pass a set of filters.

Consider an example of a partitioned Hive table. If we have a delete predicate like `part_col = '2020'`, we can just drop the matching partition to satisfy this delete. In this case, the data source should return `true` from `canDeleteWhere` and use the filters it accepts in `deleteWhere` to drop the partition. I consider this as a delete without significant effort. At the same time, if we have a delete predicate like `id = 10`, Hive tables would not be able to execute this delete using a metadata only operation without rewriting files. In that case, the data source should return `false` from `canDeleteWhere` and we should use a more sophisticated row-level API to find out which records should be removed (the API is yet to be discussed, but we need this PR as a basis).

If we decide to support subqueries and all delete use cases by simply extending the existing API, this will mean all data sources will have to implement a lot of Spark logic to determine which records changed. I don't think we want to go that way as the Spark logic to determine which records should be deleted is independent of the underlying data source. So the assumption is that Spark will execute a plan to find which records must be deleted for data sources that return `false` from `canDeleteWhere`.
### Does this PR introduce _any_ user-facing change?

Yes but it is backward compatible.

### How was this patch tested?

This PR comes with a new test.

Closes #30562 from aokolnychyi/spark-33623.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-03 09:12:30 -08:00
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
luluorta 512fb32b38 [SPARK-26218][SQL][FOLLOW UP] Fix the corner case of codegen when casting float to Integer
### What changes were proposed in this pull request?
This is a followup of [#27151](https://github.com/apache/spark/pull/27151). It fixes the same issue for the codegen path.

### Why are the changes needed?
Result corrupt.

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

### How was this patch tested?
Added Unit test.

Closes #30585 from luluorta/SPARK-26218.

Authored-by: luluorta <luluorta@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-03 14:58:56 +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
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
xuewei.linxuewei 58583f7c3f [SPARK-33619][SQL] Fix GetMapValueUtil code generation error
### What changes were proposed in this pull request?

Code Gen bug fix that introduced by SPARK-33460

```
GetMapValueUtil

s"""throw new NoSuchElementException("Key " + $eval2 + " does not exist.");"""

SHOULD BE

s"""throw new java.util.NoSuchElementException("Key " + $eval2 + " does not exist.");"""
```

And the reason why SPARK-33460 failed to detect this bug via UT,  it was because that `checkExceptionInExpression ` did not work as expect like `checkEvaluation` which will try eval expression with BOTH `CODEGEN_ONLY` and `NO_CODEGEN` mode, and in this PR, will also fix this Test bug, too.

### Why are the changes needed?

Bug Fix.

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

No.

### How was this patch tested?

Add UT and Existing UT.

Closes #30560 from leanken/leanken-SPARK-33619.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-12-02 16:10:45 +00:00
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
Liang-Chi Hsieh e518008ca9
[SPARK-33473][SQL] Extend interpreted subexpression elimination to other interpreted projections
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Unit test.

Closes #30406 from viirya/SPARK-33473.

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

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

### Why are the changes needed?

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

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

No, dev only.

### How was this patch tested?

Unit test.

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

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

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

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

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

Better ANSI SQL compliance

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

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

### How was this patch tested?

Unit test

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

Closes #30260 from gengliangwang/ansiCanCast.

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Will include new unit tests.

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

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2020-11-18 14:07:51 -08:00
Bryan Cutler 8e2a0bdce7 [SPARK-24554][PYTHON][SQL] Add MapType support for PySpark with Arrow
### What changes were proposed in this pull request?

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

### Why are the changes needed?

MapType was previous unsupported with Arrow.

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

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

### How was this patch tested?

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

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

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

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

### Why are the changes needed?

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

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

#### Benchmark

Update `SubExprEliminationBenchmark`:

Before:

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

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

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

No

### How was this patch tested?

Unit test. Benchmark manually.

Closes #30341 from viirya/SPARK-33427.

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

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

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Manual test.

Closes #30325 from wangyum/SPARK-33416.

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

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

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

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

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

### How was this patch tested?

Manually tested:

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

and unittests were also added.

Closes #30309 from HyukjinKwon/SPARK-33407.

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

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

Closes #30347 from c21/stream-test.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-17 11:18:42 +09:00
xuewei.linxuewei b5eca18af0 [SPARK-33460][SQL] Accessing map values should fail if key is not found
### What changes were proposed in this pull request?

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

### Why are the changes needed?

For ANSI mode.

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

No.

### How was this patch tested?

Added UT and Existing UT.

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

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

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

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

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

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

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

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

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

### Why are the changes needed?

Code refine.

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

No.

### How was this patch tested?

Existing test

Closes #30299 from luluorta/SPARK-33389.

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

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

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

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

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

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-16 17:57:20 +09:00
Max Gekk 4e5d2e0695 [SPARK-33394][SQL][TESTS] Throw NoSuchNamespaceException for not existing namespace in InMemoryTableCatalog.listTables()
### What changes were proposed in this pull request?
Throw `NoSuchNamespaceException` in `listTables()` of the custom test catalog `InMemoryTableCatalog` if the passed namespace doesn't exist.

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

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

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

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

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-16 07:08:21 +00:00
luluorta 156704ba0d
[SPARK-33432][SQL] SQL parser should use active SQLConf
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

output:

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

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

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

output:

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

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

No.

### How was this patch tested?

Newly and updated UTs

Closes #30357 from luluorta/SPARK-33432.

Authored-by: luluorta <luluorta@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-14 13:37:12 -08:00
Liang-Chi Hsieh 0046222a75
[SPARK-33337][SQL][FOLLOWUP] Prevent possible flakyness in SubexpressionEliminationSuite
### What changes were proposed in this pull request?

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

### Why are the changes needed?

Prevent test flakyness in SubexpressionEliminationSuite.

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

No, dev only.

### How was this patch tested?

Unit test.

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

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

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

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

### Why are the changes needed?

Revert.

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

No.

### How was this patch tested?

Existing UT.

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

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

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

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

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

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

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

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

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

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

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

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

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

== SQL ==
set;
^^^

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

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

== SQL ==
set a;
^^^

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

#### 2.4

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

new tests

Closes #30332 from yaooqinn/SPARK-33419.

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

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

### Why are the changes needed?

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

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

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

No.

### How was this patch tested?

Add test.

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

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-11-13 15:57:07 +09:00
Liang-Chi Hsieh 2c64b731ae
[SPARK-33259][SS] Disable streaming query with possible correctness issue by default
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

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

### How was this patch tested?

Unit test.

Closes #30210 from viirya/SPARK-33259.

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

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

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

### How was this patch tested?
Existing UT

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

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

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

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

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

Closes #30178 from beliefer/SPARK-33278.

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

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

### Why are the changes needed?

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

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

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

No.

### How was this patch tested?

Add test.

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

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

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

### Why are the changes needed?

For ansiMode.

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

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

### How was this patch tested?

Added UT and existing UT.

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

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-12 08:50:32 +00:00
stczwd 1eb236b936 [SPARK-32512][SQL] add alter table add/drop partition command for datasourcev2
### What changes were proposed in this pull request?
This patch is trying to add `AlterTableAddPartitionExec` and `AlterTableDropPartitionExec` with the new table partition API, defined in #28617.

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

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

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

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

existing tests and updated test in v2 write.

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

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-11 16:13:21 +09:00
Terry Kim 6d5d030957 [SPARK-33414][SQL] Migrate SHOW CREATE TABLE command to use UnresolvedTableOrView to resolve the identifier
### What changes were proposed in this pull request?

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

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

### Why are the changes needed?

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

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

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

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

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

### How was this patch tested?

Updated existing tests.

Closes #30321 from imback82/show_create_table.

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

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

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

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

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

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

Make Literal support char array.

### Why are the changes needed?

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

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

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

### How was this patch tested?

Add test.

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

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

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

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

No

### How was this patch tested?

Unit test.

Closes #30245 from viirya/SPARK-33337.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-11-10 16:17:00 -08:00
angerszhu 34f5e7ce77 [SPARK-33302][SQL] Push down filters through Expand
### What changes were proposed in this pull request?
Push down filter through expand.  For case below:
```
create table t1(pid int, uid int, sid int, dt date, suid int) using parquet;
create table t2(pid int, vs int, uid int, csid int) using parquet;

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

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

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

```

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

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

### How was this patch tested?
Added UT

Closes #30278 from AngersZhuuuu/SPARK-33302.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-10 14:40:24 +00:00
xuewei.linxuewei e3a768dd79 [SPARK-33391][SQL] element_at with CreateArray not respect one based index
### What changes were proposed in this pull request?

element_at with CreateArray not respect one based index.

repo step:

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

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

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

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

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

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

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

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

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

```

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

### Why are the changes needed?

Correctness issue.

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

No.

### How was this patch tested?

Added UT and existing UT.

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

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-10 07:23:47 +00:00
Terry Kim 90f6f39e42 [SPARK-33366][SQL] Migrate LOAD DATA command to use UnresolvedTable to resolve the identifier
### What changes were proposed in this pull request?

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Updated existing tests.

Closes #30270 from imback82/load_data_cmd.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-10 05:28:06 +00:00