Commit graph

7338 commits

Author SHA1 Message Date
Terry Kim 8792280a73 [SPARK-33575][SQL] Fix misleading exception for "ANALYZE TABLE ... FOR COLUMNS" on temporary views
### What changes were proposed in this pull request?

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

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

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

### Why are the changes needed?

To fix a misleading exception.

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

Yes, the exception thrown is changed as shown above.

### How was this patch tested?

Updated existing test.

Closes #30519 from imback82/analyze_table_message.

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

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

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

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

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

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

### How was this patch tested?

- Pass the Jenkins or GitHub Action

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

Closes #30518 from LuciferYang/SPARK-33566.

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Added UT.

Closes #30494 from maryannxue/csr-repartition.

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

Fix the wrong benchmark case name.

### Why are the changes needed?

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

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

No, dev only.

### How was this patch tested?

Unit test.

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

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

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

### Why are the changes needed?

To support Scala 2.13 in Apache Spark 3.1.

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

No.

### How was this patch tested?

Pass the GitHub Action Scala 2.13 compilation job.

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

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

This patch proposes to support subexpression elimination for interpreted predicate.

### Why are the changes needed?

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

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

No, this doesn't change user behavior.

### How was this patch tested?

Unit test and benchmark.

Closes #30497 from viirya/SPARK-33540.

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Existing tests validate there are no behavior changes.

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

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

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

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

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

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

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

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

### Why are the changes needed?

Without this pr, dynamic partition overwrite would fail

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

No.

### How was this patch tested?

added UT.

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

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

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

### Why are the changes needed?

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

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

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

No.

### How was this patch tested?

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

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

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

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Basic UT added. Manually tested with two queries:

> simple case

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

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

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

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

query.awaitTermination()
```

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

> complicated case

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

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

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

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

query.awaitTermination()
```

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

Closes #30427 from HeartSaVioR/SPARK-33224.

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

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

Note that `SHOW COLUMNS` is not yet supported for v2 tables.

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Updated existing tests.

Closes #30490 from imback82/show_columns.

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

a new test

Closes #30432 from cloud-fan/aqe.

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

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

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

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

Closes #30336 from gaborgsomogyi/SPARK-33287.

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

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

Note that `TRUNCATE TABLE` works only with v1 tables, and not supported for v2 tables.

### Why are the changes needed?

The changes allow consistent resolution behavior when resolving the table identifier. For example, the following is the current behavior:
```scala
sql("CREATE TEMPORARY VIEW t AS SELECT 1")
sql("CREATE DATABASE db")
sql("CREATE TABLE t using csv AS SELECT 1")
sql("USE db")
sql("TRUNCATE TABLE t") // Succeeds
```
With this PR, `TRUNCATE TABLE` above fails with the following:
```
org.apache.spark.sql.AnalysisException: t is a temp view not table.; line 1 pos 0
    at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
    at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTempViews$$anonfun$apply$7.$anonfun$applyOrElse$42(Analyzer.scala:866)

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

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

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

### How was this patch tested?

Updated existing tests.

Closes #30457 from imback82/truncate_table.

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

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

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

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

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

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

This patch adds predicate related benchmark to `SubExprEliminationBenchmark`.

### Why are the changes needed?

We should have a benchmark for subexpression elimination of predicate.

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

No, dev only.

### How was this patch tested?

Run benchmark locally.

Closes #30476 from viirya/SPARK-33523.

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

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

### Why are the changes needed?

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

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

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

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

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

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

Yes. This is a correctness issue.

### How was this patch tested?

Pass the UT with both Scala 2.12/2.13.

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

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

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

### Why are the changes needed?

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

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

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

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

### How was this patch tested?

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

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

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

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

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

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

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

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-23 09:00:41 +00:00
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
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
CC Highman d338af3101 [SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?

Two new options, _modifiiedBefore_  and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format.  _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.`  In order to filter file results, a new PathFilter class was derived for this purpose.  General house-keeping around classes extending PathFilter was performed for neatness.  It became apparent support was needed to handle multiple potential path filters.  Logic was introduced for this purpose and the associated tests written.

### Why are the changes needed?

When loading files from a data source, there can often times be thousands of file within a respective file path.  In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point.  This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered.  This saves a ton of time automatically and reduces significant complexity managing this in code.

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

This PR introduces an option that can be used with batch-based Spark file data sources.  A documentation update was made to reflect an example and usage of the new data source option.

**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`

_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`

_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`

### How was this patch tested?

A handful of unit tests were added to support the positive, negative, and edge case code paths.

It's also live in a handful of our Databricks dev environments.  (quoted from cchighman)

Closes #30411 from HeartSaVioR/SPARK-31962.

Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-23 08:30:41 +09: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
Jungtaek Lim (HeartSaVioR) 67c6ed9068 [SPARK-33223][SS][FOLLOWUP] Clarify the meaning of "number of rows dropped by watermark" in SS UI page
### What changes were proposed in this pull request?

This PR fixes the representation to clarify the meaning of "number of rows dropped by watermark" in SS UI page.

### Why are the changes needed?

`Aggregated Number Of State Rows Dropped By Watermark` says that the dropped rows are from the state, whereas they're not. We say "evicted from the state" for the case, which is "normal" to emit outputs and reduce memory usage of the state.

The metric actually represents the number of "input" rows dropped by watermark, and the meaning of "input" is relative to the "stateful operator". That's a bit confusing as we normally think "input" as "input from source" whereas it's not.

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

Yes, UI element & tooltip change.

### How was this patch tested?

Only text change in UI, so we know how thing will be changed intuitively.

Closes #30439 from HeartSaVioR/SPARK-33223-FOLLOWUP.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-21 10:27:00 +09:00
Chao Sun 2479778934
[SPARK-33492][SQL] DSv2: Append/Overwrite/ReplaceTable should invalidate cache
### What changes were proposed in this pull request?

This adds changes in the following places:
- logic to also refresh caches referencing the target table in v2 `AppendDataExec`, `OverwriteByExpressionExec`, `OverwritePartitionsDynamicExec`, as well as their v1 fallbacks `AppendDataExecV1` and `OverwriteByExpressionExecV1`.
- logic to invalidate caches referencing the target table in v2 `ReplaceTableAsSelectExec` and its atomic version `AtomicReplaceTableAsSelectExec`. These are only supported in v2 at the moment though.

In addition to the above, in order to test the v1 write fallback behavior, I extended `InMemoryTableWithV1Fallback` to also support batch reads.

### Why are the changes needed?

Currently in DataSource v2 we don't refresh or invalidate caches referencing the target table when the table content is changed by operations such as append, overwrite, or replace table. This is different from DataSource v1, and could potentially cause data correctness issue if the staled caches are queried later.

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

Yes. Now When a data source v2 is cached (either directly or indirectly), all the relevant caches will be refreshed or invalidated if the table is replaced.

### How was this patch tested?

Added unit tests for the new code path.

Closes #30429 from sunchao/SPARK-33492.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-20 14:59:56 -08:00
Max Gekk 870d409533 [SPARK-32512][SQL][TESTS][FOLLOWUP] Remove duplicate tests for ALTER TABLE .. PARTITIONS from DataSourceV2SQLSuite
### What changes were proposed in this pull request?
Remove tests from `DataSourceV2SQLSuite` that were copied to `AlterTablePartitionV2SQLSuite` by https://github.com/apache/spark/pull/29339.

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

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

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

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

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

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

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

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

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

No.

### How was this patch tested?

Add test.

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

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

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

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

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

### How was this patch tested?
Unit test

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

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-19 13:29:01 +00:00
Prakhar Jain 0b0fb70b09 [SPARK-33400][SQL] Normalize sameOrderExpressions in SortOrder to avoid unnecessary sort operations
### What changes were proposed in this pull request?
This pull request tries to normalize the SortOrder properly to prevent unnecessary sort operators. Currently the sameOrderExpressions are not normalized as part of AliasAwareOutputOrdering.

Example: consider this join of three tables:

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

The plan for this looks like:

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

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

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

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

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

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

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-19 06:25:37 +00:00
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
Gengliang Wang 9a4c79073b [SPARK-33354][SQL] New explicit cast syntax rules in ANSI mode
### What changes were proposed in this pull request?

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

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

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

Better ANSI SQL compliance

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

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

### How was this patch tested?

Unit test

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

Closes #30260 from gengliangwang/ansiCanCast.

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Will include new unit tests.

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

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

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

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

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-11-18 12:39:00 -08:00
Bryan Cutler 8e2a0bdce7 [SPARK-24554][PYTHON][SQL] Add MapType support for PySpark with Arrow
### What changes were proposed in this pull request?

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

### Why are the changes needed?

MapType was previous unsupported with Arrow.

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

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

### How was this patch tested?

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

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

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

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

### Why are the changes needed?

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

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

Yes, update user-facing doc.

### How was this patch tested?

Unit test.

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

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

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

### Why are the changes needed?

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

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

#### Benchmark

Update `SubExprEliminationBenchmark`:

Before:

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

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

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

No

### How was this patch tested?

Unit test. Benchmark manually.

Closes #30341 from viirya/SPARK-33427.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-17 14:29:37 +00:00
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
Prakhar Jain f5e3302840 [SPARK-33399][SQL] Normalize output partitioning and sortorder with respect to aliases to avoid unneeded exchange/sort nodes
### What changes were proposed in this pull request?
This pull request tries to remove unneeded exchanges/sorts by normalizing the output partitioning and sortorder information correctly with respect to aliases.

Example: consider this join of three tables:

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

The plan for this looks like:

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

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

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

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

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

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

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

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

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

### Why are the changes needed?

For ANSI mode.

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

No.

### How was this patch tested?

Added UT and Existing UT.

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

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

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

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

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

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

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

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

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

### Why are the changes needed?

Code refine.

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

No.

### How was this patch tested?

Existing test

Closes #30299 from luluorta/SPARK-33389.

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

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

### Why are the changes needed?

Potential regression bug.

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

No.

### How was this patch tested?

Existing UT.

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

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-16 11:57:50 +00:00
Liang-Chi Hsieh 10b011f837 [SPARK-33456][SQL][TEST][FOLLOWUP] Fix SUBEXPRESSION_ELIMINATION_ENABLED config name
### What changes were proposed in this pull request?

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

### Why are the changes needed?

`CONFIG_DIM` should use config name's key.

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

No, dev only.

### How was this patch tested?

Unit test.

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

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

No, dev only.

### How was this patch tested?

Unit tests.

Closes #30381 from viirya/SPARK-33456.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-16 15:47:35 +09:00
Liang-Chi Hsieh eea846b895
[SPARK-33455][SQL][TEST] Add SubExprEliminationBenchmark for benchmarking subexpression elimination
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

No, dev only.

### How was this patch tested?

Unit test.

Closes #30379 from viirya/SPARK-33455.

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

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

### Why are the changes needed?

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

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

output:

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

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

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

output:

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

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

No.

### How was this patch tested?

Newly and updated UTs

Closes #30357 from luluorta/SPARK-33432.

Authored-by: luluorta <luluorta@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-14 13:37:12 -08:00