Commit graph

9197 commits

Author SHA1 Message Date
Kent Yao 7b4b29e8d9
[SPARK-31131][SQL] Remove the unnecessary config spark.sql.legacy.timeParser.enabled
### What changes were proposed in this pull request?

spark.sql.legacy.timeParser.enabled should be removed from SQLConf and the migration guide
spark.sql.legacy.timeParsePolicy is the right one

### Why are the changes needed?

fix doc

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

no
### How was this patch tested?

Pass the jenkins

Closes #27889 from yaooqinn/SPARK-31131.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-12 09:24:49 -07:00
Wenchen Fan 77c49cb702 [SPARK-31124][SQL] change the default value of minPartitionNum in AQE
### What changes were proposed in this pull request?

AQE has a perf regression when using the default settings: if we coalesce the shuffle partitions into one or few partitions, we may leave many CPU cores idle and the perf is worse than with AQE off (which leverages all CPU cores).

Technically, this is not a bad thing. If there are many queries running at the same time, it's better to coalesce shuffle partitions into fewer partitions. However, the default settings of AQE should try to avoid any perf regression as possible as we can.

This PR changes the default value of minPartitionNum when coalescing shuffle partitions, to be `SparkContext.defaultParallelism`, so that AQE can leverage all the CPU cores.

### Why are the changes needed?

avoid AQE perf regression

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

No

### How was this patch tested?

existing tests

Closes #27879 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-12 21:28:24 +08:00
yi.wu feb9b9e771 [SPARK-31010][SQL][FOLLOW-UP] Give an example for typed Scala UDF in error message
### What changes were proposed in this pull request?

In the error message, adding an example for typed Scala UDF.

### Why are the changes needed?

Help user to know how to migrate to typed Scala UDF.

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

No, it's a new error message in Spark 3.0.

### How was this patch tested?

Pass Jenkins.

Closes #27884 from Ngone51/spark_31010_followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-12 21:16:02 +09:00
Kent Yao 18f2730874 [SPARK-31066][SQL][TEST-HIVE1.2] Disable useless and uncleaned hive SessionState initialization parts
### What changes were proposed in this pull request?

As a common usage and according to the spark doc, users may often just copy their `hive-site.xml` to Spark directly from hive projects. Sometimes, the config file is not that clean for spark and may cause some side effects.

for example, `hive.session.history.enabled` will create a log for the hive jobs but useless for spark and also it will not be deleted on JVM exit.

this pr
 1) disable `hive.session.history.enabled` explicitly to disable creating `hive_job_log` file, e.g.
```
Hive history file=/var/folders/01/h81cs4sn3dq2dd_k4j6fhrmc0000gn/T//kentyao/hive_job_log_79c63b29-95a4-4935-a9eb-2d89844dfe4f_493861201.txt
```
2) set `hive.execution.engine` to `spark` explicitly in case the config is `tez` and casue uneccesary problem like this:

```
Exception in thread "main" java.lang.NoClassDefFoundError: org/apache/tez/dag/api/SessionNotRunning
	at org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:529)
```

### Why are the changes needed?

reduce overhead of internal complexity and users' hive cognitive load for  running spark

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

yes, `hive_job_log` file will not be created even enabled, and will not try to initialize tez kinds of stuff
### How was this patch tested?

add ut and verify manually

Closes #27827 from yaooqinn/SPARK-31066.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-12 18:13:52 +08:00
Wenchen Fan 8efb71013d
[SPARK-31091] Revert SPARK-24640 Return NULL from size(NULL) by default
### What changes were proposed in this pull request?

This PR reverts https://github.com/apache/spark/pull/26051 and https://github.com/apache/spark/pull/26066

### Why are the changes needed?

There is no standard requiring that `size(null)` must return null, and returning -1 looks reasonable as well. This is kind of a cosmetic change and we should avoid it if it breaks existing queries. This is similar to reverting TRIM function parameter order change.

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

Yes, change the behavior of `size(null)` back to be the same as 2.4.

### How was this patch tested?

N/A

Closes #27834 from cloud-fan/revert.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-11 09:55:24 -07:00
Wenchen Fan 5be0d04f16 [SPARK-31117][SQL][TEST] reduce the test time of DateTimeUtilsSuite
### What changes were proposed in this pull request?

`DateTimeUtilsSuite.daysToMicros and microsToDays` takes 30 seconds, which is too long for a UT.

This PR changes the test to check random data, to reduce testing time. Now this test takes 1 second.

### Why are the changes needed?

make test faster

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

no

### How was this patch tested?

N/A

Closes #27873 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-11 23:47:13 +08:00
Maxim Gekk 3d3e366aa8 [SPARK-31076][SQL] Convert Catalyst's DATE/TIMESTAMP to Java Date/Timestamp via local date-time
### What changes were proposed in this pull request?
In the PR, I propose to change conversion of java.sql.Timestamp/Date values to/from internal values of Catalyst's TimestampType/DateType before cutover day `1582-10-15` of Gregorian calendar. I propose to construct local date-time from microseconds/days since the epoch. Take each date-time component `year`, `month`, `day`, `hour`, `minute`, `second` and `second fraction`, and construct java.sql.Timestamp/Date using the extracted components.

### Why are the changes needed?
This will rebase underlying time/date offset in the way that collected java.sql.Timestamp/Date values will have the same local time-date component as the original values in Gregorian calendar.

Here is the example which demonstrates the issue:
```sql
scala> sql("select date '1100-10-10'").collect()
res1: Array[org.apache.spark.sql.Row] = Array([1100-10-03])
```

### Does this PR introduce any user-facing change?
Yes, after the changes:
```sql
scala> sql("select date '1100-10-10'").collect()
res0: Array[org.apache.spark.sql.Row] = Array([1100-10-10])
```

### How was this patch tested?
By running `DateTimeUtilsSuite`, `DateFunctionsSuite` and `DateExpressionsSuite`.

Closes #27807 from MaxGekk/rebase-timestamp-before-1582.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-11 20:53:56 +08:00
Kent Yao 2b46662bd0 [SPARK-31111][SQL][TESTS] Fix interval output issue in ExtractBenchmark
### What changes were proposed in this pull request?

fix the error caused by interval output in ExtractBenchmark
### Why are the changes needed?

fix a bug in the test

```scala
[info]   Running case: cast to interval
[error] Exception in thread "main" org.apache.spark.sql.AnalysisException: Cannot use interval type in the table schema.;;
[error] OverwriteByExpression RelationV2[] noop-table, true, true
[error] +- Project [(subtractdates(cast(cast(id#0L as timestamp) as date), -719162) + subtracttimestamps(cast(id#0L as timestamp), -30610249419876544)) AS ((CAST(CAST(id AS TIMESTAMP) AS DATE) - DATE '0001-01-01') + (CAST(id AS TIMESTAMP) - TIMESTAMP '1000-01-01 01:02:03.123456'))#2]
[error]    +- Range (1262304000, 1272304000, step=1, splits=Some(1))
[error]
[error] 	at org.apache.spark.sql.catalyst.util.TypeUtils$.failWithIntervalType(TypeUtils.scala:106)
[error] 	at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$25(CheckAnalysis.scala:389)
[error] 	at org.a
```
### Does this PR introduce any user-facing change?

no

### How was this patch tested?

re-run benchmark

Closes #27867 from yaooqinn/SPARK-31111.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-11 20:13:59 +08:00
Liang-Chi Hsieh 15557a7d05 [SPARK-31071][SQL] Allow annotating non-null fields when encoding Java Beans
### What changes were proposed in this pull request?

When encoding Java Beans to Spark DataFrame, respecting `javax.annotation.Nonnull` and producing non-null fields.

### Why are the changes needed?

When encoding Java Beans to Spark DataFrame, non-primitive types are encoded as nullable fields. Although It works for most cases, it can be an issue under a few situations, e.g. the one described in the JIRA ticket when saving DataFrame to Avro format with non-null field.

We should allow Spark users more flexibility when creating Spark DataFrame from Java Beans. Currently, Spark users cannot create DataFrame with non-nullable fields in the schema from beans with non-nullable properties.

Although it is possible to project top-level columns with SQL expressions like `AssertNotNull` to make it non-null, for nested fields it is more tricky to do it similarly.

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

Yes. After this change, Spark users can use `javax.annotation.Nonnull` to annotate non-null fields in Java Beans when encoding beans to Spark DataFrame.

### How was this patch tested?

Added unit test.

Closes #27851 from viirya/SPARK-31071.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-11 18:27:48 +08:00
Yuanjian Li 3493162c78 [SPARK-31030][SQL] Backward Compatibility for Parsing and formatting Datetime
### What changes were proposed in this pull request?
In Spark version 2.4 and earlier, datetime parsing, formatting and conversion are performed by using the hybrid calendar (Julian + Gregorian).
Since the Proleptic Gregorian calendar is de-facto calendar worldwide, as well as the chosen one in ANSI SQL standard, Spark 3.0 switches to it by using Java 8 API classes (the java.time packages that are based on ISO chronology ). The switching job is completed in SPARK-26651.
But after the switching, there are some patterns not compatible between Java 8 and Java 7, Spark needs its own definition on the patterns rather than depends on Java API.
In this PR, we achieve this by writing the document and shadow the incompatible letters. See more details in [SPARK-31030](https://issues.apache.org/jira/browse/SPARK-31030)

### Why are the changes needed?
For backward compatibility.

### Does this PR introduce any user-facing change?
No.
After we define our own datetime parsing and formatting patterns, it's same to old Spark version.

### How was this patch tested?
Existing and new added UT.
Locally document test:
![image](https://user-images.githubusercontent.com/4833765/76064100-f6acc280-5fc3-11ea-9ef7-82e7dc074205.png)

Closes #27830 from xuanyuanking/SPARK-31030.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-11 14:11:13 +08:00
Wenchen Fan d5f5720efa [SPARK-31070][SQL] make skew join split skewed partitions more evenly
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
  7. If you want to add a new configuration, please read the guideline first for naming configurations in
     'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
-->

### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->
There are two problems when splitting skewed partitions:
1. It's impossible that we can't split the skewed partition, then we shouldn't create a skew join.
2. When splitting, it's possible that we create a partition for very small amount of data..

This PR fixes them
1. don't create `PartialReducerPartitionSpec` if we can't split.
2. merge small partitions to the previous partition.
### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->
make skew join split skewed partitions more evenly

### Does this PR introduce any user-facing change?
<!--
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If no, write 'No'.
-->
no

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
updated test

Closes #27833 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2020-03-10 21:50:44 -07:00
yi.wu 34be83e08b
[SPARK-31037][SQL][FOLLOW-UP] Replace legacy ReduceNumShufflePartitions with CoalesceShufflePartitions in comment
### What changes were proposed in this pull request?

Replace legacy `ReduceNumShufflePartitions` with `CoalesceShufflePartitions` in comment.

### Why are the changes needed?

Rule `ReduceNumShufflePartitions` has renamed to `CoalesceShufflePartitions`, we should update related comment as well.

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

No.

### How was this patch tested?

N/A.

Closes #27865 from Ngone51/spark_31037_followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-10 11:09:36 -07:00
Kent Yao 3bd6ebff81 [SPARK-30189][SQL] Interval from year-month/date-time string should handle whitespaces
### What changes were proposed in this pull request?

Currently, we parse interval from multi units strings or from date-time/year-month pattern strings, the former handles all whitespace, the latter not or even spaces.

### Why are the changes needed?

behavior consistency

### Does this PR introduce any user-facing change?
yes, interval in date-time/year-month like
```
select interval '\n-\t10\t 12:34:46.789\t' day to second
-- !query 126 schema
struct<INTERVAL '-10 days -12 hours -34 minutes -46.789 seconds':interval>
-- !query 126 output
-10 days -12 hours -34 minutes -46.789 seconds
```
is valid now.

### How was this patch tested?

add ut.

Closes #26815 from yaooqinn/SPARK-30189.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-10 22:08:58 +08:00
Terry Kim 294f6056fa [SPARK-31078][SQL] Respect aliases in output ordering
### What changes were proposed in this pull request?

Currently, in the following scenario, an unnecessary `Sort` node is introduced:
```scala
withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0") {
  val df = (0 until 20).toDF("i").as("df")
  df.repartition(8, df("i")).write.format("parquet")
    .bucketBy(8, "i").sortBy("i").saveAsTable("t")
  val t1 = spark.table("t")
  val t2 = t1.selectExpr("i as ii")
  t1.join(t2, t1("i") === t2("ii")).explain
}
```
```
== Physical Plan ==
*(3) SortMergeJoin [i#8], [ii#10], Inner
:- *(1) Project [i#8]
:  +- *(1) Filter isnotnull(i#8)
:     +- *(1) ColumnarToRow
:        +- FileScan parquet default.t[i#8] Batched: true, DataFilters: [isnotnull(i#8)], Format: Parquet, Location: InMemoryFileIndex[file:/..., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int>, SelectedBucketsCount: 8 out of 8
+- *(2) Sort [ii#10 ASC NULLS FIRST], false, 0    <==== UNNECESSARY
   +- *(2) Project [i#8 AS ii#10]
      +- *(2) Filter isnotnull(i#8)
         +- *(2) ColumnarToRow
            +- FileScan parquet default.t[i#8] Batched: true, DataFilters: [isnotnull(i#8)], Format: Parquet, Location: InMemoryFileIndex[file:/..., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int>, SelectedBucketsCount: 8 out of 8
```
Notice that `Sort [ii#10 ASC NULLS FIRST], false, 0` is introduced even though the underlying data is already sorted. This is because `outputOrdering` doesn't handle aliases correctly. This PR proposes to fix this issue.

### Why are the changes needed?

To better handle aliases in `outputOrdering`.

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

Yes, now with the fix, the `explain` prints out the following:
```
== Physical Plan ==
*(3) SortMergeJoin [i#8], [ii#10], Inner
:- *(1) Project [i#8]
:  +- *(1) Filter isnotnull(i#8)
:     +- *(1) ColumnarToRow
:        +- FileScan parquet default.t[i#8] Batched: true, DataFilters: [isnotnull(i#8)], Format: Parquet, Location: InMemoryFileIndex[file:/..., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int>, SelectedBucketsCount: 8 out of 8
+- *(2) Project [i#8 AS ii#10]
   +- *(2) Filter isnotnull(i#8)
      +- *(2) ColumnarToRow
         +- FileScan parquet default.t[i#8] Batched: true, DataFilters: [isnotnull(i#8)], Format: Parquet, Location: InMemoryFileIndex[file:/..., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int>, SelectedBucketsCount: 8 out of 8
```

### How was this patch tested?

Tests added.

Closes #27842 from imback82/alias_aware_sort_order.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-10 20:15:48 +08:00
Eric Wu 15df2a3f40 [SPARK-31079][SQL] Logging QueryExecutionMetering in RuleExecutor logger
### What changes were proposed in this pull request?
RuleExecutor already support metering for analyzer/optimizer rules. By providing such information in `PlanChangeLogger`, user can get more information when debugging rule changes .

This PR enhanced `PlanChangeLogger` to display RuleExecutor metrics. This can be easily done by calling the existing API `resetMetrics` and `dumpTimeSpent`, but there might be conflicts if user is also collecting total metrics of a sql job. Thus I introduced `QueryExecutionMetrics`, as the snapshot of `QueryExecutionMetering`, to better support this feature.

Information added to `PlanChangeLogger`
```
=== Metrics of Executed Rules ===
Total number of runs: 554
Total time: 0.107756568 seconds
Total number of effective runs: 11
Total time of effective runs: 0.047615486 seconds
```

### Why are the changes needed?
Provide better plan change debugging user experience

### Does this PR introduce any user-facing change?
Only add more debugging info of `planChangeLog`, default log level is TRACE.

### How was this patch tested?
Update existing tests to verify the new logs

Closes #27846 from Eric5553/ExplainRuleExecMetrics.

Authored-by: Eric Wu <492960551@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-10 19:08:59 +08:00
HyukjinKwon 815c7929c2
[SPARK-31065][SQL] Match schema_of_json to the schema inference of JSON data source
### What changes were proposed in this pull request?

This PR proposes two things:

1. Convert `null` to `string` type during schema inference of `schema_of_json` as JSON datasource does. This is a bug fix as well because `null` string is not the proper DDL formatted string and it is unable for SQL parser to recognise it as a type string. We should match it to JSON datasource and return a string type so `schema_of_json` returns a proper DDL formatted string.

2. Let `schema_of_json` respect `dropFieldIfAllNull` option during schema inference.

### Why are the changes needed?

To let `schema_of_json` return a proper DDL formatted string, and respect `dropFieldIfAllNull` option.

### Does this PR introduce any user-facing change?
Yes, it does.

```scala
import collection.JavaConverters._
import org.apache.spark.sql.functions._

spark.range(1).select(schema_of_json(lit("""{"id": ""}"""))).show()
spark.range(1).select(schema_of_json(lit("""{"id": "a", "drop": {"drop": null}}"""), Map("dropFieldIfAllNull" -> "true").asJava)).show(false)
```

**Before:**

```
struct<id:null>
struct<drop:struct<drop:null>,id:string>
```

**After:**

```
struct<id:string>
struct<id:string>
```

### How was this patch tested?

Manually tested, and unittests were added.

Closes #27854 from HyukjinKwon/SPARK-31065.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-10 00:33:32 -07:00
maryannxue de6d9e4307 [SPARK-31096][SQL] Replace Array with Seq in AQE CustomShuffleReaderExec
### What changes were proposed in this pull request?
This PR changes the type of `CustomShuffleReaderExec`'s `partitionSpecs` from `Array` to `Seq`, since `Array` compares references not values for equality, which could lead to potential plan reuse problem.

### Why are the changes needed?
Unlike `Seq`, `Array` compares references not values for equality, which could lead to potential plan reuse problem.

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

### How was this patch tested?
Passes existing UTs.

Closes #27857 from maryannxue/aqe-customreader-fix.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-10 14:15:44 +08:00
Yuchen Huo a22994333a [SPARK-30902][SQL][FOLLOW-UP] Allow ReplaceTableAsStatement to have none provider
### What changes were proposed in this pull request?

This is a follow up for https://github.com/apache/spark/pull/27650 where allow None provider for create table. Here we are doing the same thing for ReplaceTable.

### Why are the changes needed?

Although currently the ASTBuilder doesn't seem to allow `replace` without `USING` clause. This would allow `DataFrameWriterV2` to use the statements instead of commands directly.

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

No.

### How was this patch tested?

Existing tests

Closes #27838 from yuchenhuo/SPARK-30902.

Authored-by: Yuchen Huo <yuchen.huo@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-10 11:37:31 +08:00
yi.wu ef51ff9dc8 [SPARK-31082][CORE] MapOutputTrackerMaster.getMapLocation should handle last mapIndex correctly
### What changes were proposed in this pull request?

In `getMapLocation`, change the condition from `...endMapIndex < statuses.length` to `...endMapIndex <= statuses.length`.

### Why are the changes needed?

`endMapIndex` is exclusive, we should include it when comparing to `statuses.length`. Otherwise, we can't get the location for last mapIndex.

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

No.

### How was this patch tested?

Updated existed test.

Closes #27850 from Ngone51/fix_getmaploction.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-09 15:53:34 +08:00
Wenchen Fan 1aa184763a
[SPARK-31053][SQL] mark connector APIs as Evolving
### What changes were proposed in this pull request?

The newly added catalog APIs are marked as Experimental but other DS v2 APIs are marked as Evolving.

This PR makes it consistent and mark all Connector APIs as Evolving.

### Why are the changes needed?

For consistency.

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

no

### How was this patch tested?

N/A

Closes #27811 from cloud-fan/tag.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-08 11:41:09 -07:00
beliefer f8a3730fd7 [SPARK-30841][SQL][DOC][FOLLOW-UP] Add version information to the configuration of SQL
### What changes were proposed in this pull request?
This PR follows https://github.com/apache/spark/pull/27691, https://github.com/apache/spark/pull/27730 and https://github.com/apache/spark/pull/27770
I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.sql.redaction.options.regex | 2.2.2 | SPARK-23850 | 6a55d8b03053e616dcacb79cd2c29a06d219dc32#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.redaction.string.regex | 2.3.0 | SPARK-22791 | 28315714ddef3ddcc192375e98dd5207cf4ecc98#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.function.concatBinaryAsString | 2.3.0 | SPARK-22771 | f2b3525c17d660cf6f082bbafea8632615b4f58e#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.function.eltOutputAsString | 2.3.0 | SPARK-22937 | bf853018cabcd3b3abf84bfe534d2981020b4a71#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.sources.validatePartitionColumns | 3.0.0 | SPARK-26263 | 5a140b7844936cf2b65f08853b8cfd8c499d4f13#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.streaming.continuous.epochBacklogQueueSize | 3.0.0 | SPARK-24063 | c4bbfd177b4e7cb46f47b39df9fd71d2d9a12c6d#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.streaming.continuous.executorQueueSize | 2.3.0 | SPARK-22789 | 8941a4abcada873c26af924e129173dc33d66d71#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.streaming.continuous.executorPollIntervalMs | 2.3.0 | SPARK-22789 | 8941a4abcada873c26af924e129173dc33d66d71#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.sources.useV1SourceList | 3.0.0 | SPARK-28747 | cb06209fc908bac6ce6a8f20653865489773cbc3#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.streaming.disabledV2Writers | 2.3.1 | SPARK-23196 | 588b9694c1967ff45774431441e84081ee6eb515#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.streaming.disabledV2MicroBatchReaders | 2.4.0 | SPARK-23362 | 0a73aa31f41c83503d5d99eff3c9d7b406014ab3#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.sources.partitionOverwriteMode | 2.3.0 | SPARK-20236 | b96248862589bae1ddcdb14ce4c802789a001306#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.storeAssignmentPolicy | 3.0.0 | SPARK-28730 | 895c90b582cc2b2667241f66d5b733852aeef9eb#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.ansi.enabled | 3.0.0 | SPARK-30125 | d9b30694122f8716d3acb448638ef1e2b96ebc7a#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.execution.sortBeforeRepartition | 2.1.4 | SPARK-23207 and SPARK-22905 and SPARK-24564 and SPARK-25114 | 4d2d3d47e00e78893b1ecd5a9a9070adc5243ac9#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.nestedSchemaPruning.enabled | 2.4.1 | SPARK-4502 | dfcff38394929970fee454c69864d0e10d59f8d4#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.serializer.nestedSchemaPruning.enabled | 3.0.0 | SPARK-26837 | 0f2c0b53e8fb18c86c67b5dd679c006db93f94a5#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.expression.nestedPruning.enabled | 3.0.0 | SPARK-27707 | 127bc899ae78d73332a87f0972b5db3c9936c1f1#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.execution.topKSortFallbackThreshold | 2.4.0 | SPARK-24193 | 8a837bf4f3f2758f7825d2362cf9de209026651a#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.csv.parser.columnPruning.enabled | 2.4.0 | SPARK-24244 and SPARK-24368 | 64fad0b519cf35b8c0a0dec18dd3df9488a5ed25#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.repl.eagerEval.enabled | 2.4.0 | SPARK-24215 | 6a0b77a55d53e74ac0a0892556c3a7a933474948#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.repl.eagerEval.maxNumRows | 2.4.0 | SPARK-24215 | 6a0b77a55d53e74ac0a0892556c3a7a933474948#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.repl.eagerEval.truncate | 2.4.0 | SPARK-24215 | 6a0b77a55d53e74ac0a0892556c3a7a933474948#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.codegen.aggregate.fastHashMap.capacityBit | 2.4.0 | SPARK-24978 | 6193a202aab0271b4532ee4b740318290f2c44a1#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.avro.compression.codec | 2.4.0 | SPARK-24881 | 0a0f68bae6c0a1bf30184b1e9ac6bf3805bd7511#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.avro.deflate.level | 2.4.0 | SPARK-24881 | 0a0f68bae6c0a1bf30184b1e9ac6bf3805bd7511#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.sizeOfNull | 2.4.0 | SPARK-24605 | d08f53dc61f662f5291f71bcbe1a7b9f531a34d2#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.replaceDatabricksSparkAvro.enabled | 2.4.0 | SPARK-25129 | ac0174e55af2e935d41545721e9f430c942b3a0c#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.setopsPrecedence.enabled | 2.4.0 | SPARK-24966 | 73dd6cf9b558f9d752e1f3c13584344257ad7863#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.exponentLiteralAsDecimal.enabled | 3.0.0 | SPARK-29956 | 87ebfaf003fcd05a7f6d23b3ecd4661409ce5f2f#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.allowNegativeScaleOfDecimal | 3.0.0 | SPARK-30812 | b76bc0b1b8b2abd00a84f805af90ca4c5925faaa#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.createHiveTableByDefault.enabled | 3.0.0 | SPARK-30098 | 58be82ad4b98fc17e821e916e69e77a6aa36209d#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.integralDivide.returnBigint | 3.0.0 | SPARK-25457 | 47d6e80a2e64823fabb596503fb6a6cc6f51f713#diff-9a6b543db706f1a90f790783d6930a13 | Exists in branch-3.0 branch, but the pom.xml file corresponding to the commit log is 2.5.0-SNAPSHOT
spark.sql.legacy.bucketedTableScan.outputOrdering | 3.0.0 | SPARK-28595 | 469423f33887a966aaa33eb75f5e7974a0a97beb#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.parser.havingWithoutGroupByAsWhere | 2.4.1 | SPARK-25708 | 3dba5d41f1a66ae5eb08404d103284110c45a351#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.json.allowEmptyString.enabled | 3.0.0 | SPARK-25040 | d3de7568f32e298442f07b0a28b2c906de72c797#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.createEmptyCollectionUsingStringType | 3.0.0 | SPARK-30790 | 8ab6ae3ede96adb093347470a5cbbf17fe8c04e9#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.allowUntypedScalaUDF | 3.0.0 | SPARK-26580 | bc30a07ce262840c99a752db4fbd3a423f652017#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.truncateTable.ignorePermissionAcl.enabled | 2.4.6 | SPARK-30312 | 830a4ec59b86253f18eb7dfd6ed0bbe0d7920e5b#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue | 3.0.0 | SPARK-26085 | ab2eafb3cdc7631452650c6cac03a92629255347#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.debug.maxToStringFields | 3.0.0 | SPARK-26066 | 81550b38e43fb20f89f529d2127575c71a54a538#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.maxPlanStringLength | 3.0.0 | SPARK-26103 | 812ad5546148d2194ab0e4230ee85b8f6a5be2fb#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.setCommandRejectsSparkCoreConfs | 3.0.0 | SPARK-26060 | 1ab3d3e474ce2e36d58aea8ad09fb61f0c73e5c5#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.datetime.java8API.enabled | 3.0.0 | SPARK-27008 | 52671d631d2a64ed1cfa0c6e01168908faf92df8#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.sources.binaryFile.maxLength | 3.0.0 | SPARK-27588 | 618d6bff71073c8c93501ab7392c3cc579730f0b#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.typeCoercion.datetimeToString.enabled | 3.0.0 | SPARK-27638 | 83d289eef492de8c7f3e5145f9bd75431608b500#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.defaultCatalog | 3.0.0 | SPARK-29753 | 942753a44beeae5f0142ceefa307e90cbc1234c5#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.catalog.$SESSION_CATALOG_NAME | 3.0.0 | SPARK-29412 | 9407fba0375675d6ee6461253f3b8230e8d67509#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.legacy.doLooseUpcast | 3.0.0 | SPARK-30812 | b76bc0b1b8b2abd00a84f805af90ca4c5925faaa#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.ctePrecedencePolicy | 3.0.0 | SPARK-30829 | 00943be81afbca6be13e1e72b24536cd98a788d6#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.timeParserPolicy | 3.1.0 | SPARK-30668 | 7db0af578585ecaeee9fd23f8189292289b52a97#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.followThreeValuedLogicInArrayExists | 3.0.0 | SPARK-30812 | b76bc0b1b8b2abd00a84f805af90ca4c5925faaa#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.maven.additionalRemoteRepositories | 3.0.0 | SPARK-29175 | 3d7359ad4202067b26a199657b6a3e1f38be0e4d#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.legacy.fromDayTimeString.enabled | 3.0.0 | SPARK-29864 and SPARK-29920 | e933539cdd557297daf97ff5e532a3f098896979#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.notReserveProperties | 3.0.0 | SPARK-30812 | b76bc0b1b8b2abd00a84f805af90ca4c5925faaa#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.addSingleFileInAddFile | 3.0.0 | SPARK-30234 | 8a8d1fbb10af6da481f26831cd519ef46ccbce6c#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.mssqlserver.numericMapping.enabled | 2.4.5 | SPARK-28152 | 69de7f31c37a7e0298e66cc814afc1b0aa948bbb#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.csv.filterPushdown.enabled | 3.0.0 | SPARK-30323 | 4e50f0291f032b4a5c0b46ed01fdef14e4cbb050#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.addPartitionInBatch.size | 3.0.0 | SPARK-29938 | 5ccbb38a71890b114c707279e7395d1f6284ebfd#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.timeParser.enabled | 3.0.0 | SPARK-30668 | 92f57237871400ab9d499e1174af22a867c01988#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.allowDuplicatedMapKeys | 3.0.0 | SPARK-25829 | 33329caa81827a245b84158b13234b88a4746e56#diff-9a6b543db706f1a90f790783d6930a13 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27829 from beliefer/add-version-to-sql-config-part-four.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-08 12:33:02 +09:00
DB Tsai 7911f95202 [SPARK-31064][SQL] New Parquet Predicate Filter APIs with multi-part Identifier Support
### What changes were proposed in this pull request?
Parquet's org.apache.parquet.filter2.predicate.FilterApi uses `dots` as separators to split the column name into multi-parts of nested fields. The drawback is this causes issues when the field name contains `dots`.

The new APIs that will be added will take array of string directly for multi-parts of nested fields, so no confusion as using `dots` as separators.

### Why are the changes needed?
To support nested predicate pushdown and predicate pushdown for columns containing `dots`.

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

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

Closes #27824 from dbtsai/SPARK-31064.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-03-06 21:09:24 +00:00
iRakson cba17e07e9 [SPARK-30899][SQL] CreateArray/CreateMap's data type should not depend on SQLConf.get
### What changes were proposed in this pull request?
Introduced a new parameter `emptyCollection` for `CreateMap` and `CreateArray` functiion to remove dependency on SQLConf.get.

### Why are the changes needed?
This allows to avoid the issue when the configuration change between different phases of planning, and this can silently break a query plan which can lead to crashes or data corruption.

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

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

Closes #27657 from iRakson/SPARK-30899.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-06 16:45:06 +08:00
Takeshi Yamamuro 71c73d58f6 [SPARK-30279][SQL] Support 32 or more grouping attributes for GROUPING_ID
### What changes were proposed in this pull request?

This pr intends to support 32 or more grouping attributes for GROUPING_ID. In the current master, an integer overflow can occur to compute grouping IDs;
e75d9afb2f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala (L613)

For example, the query below generates wrong grouping IDs in the master;
```

scala> val numCols = 32 // or, 31
scala> val cols = (0 until numCols).map { i => s"c$i" }
scala> sql(s"create table test_$numCols (${cols.map(c => s"$c int").mkString(",")}, v int) using parquet")
scala> val insertVals = (0 until numCols).map { _ => 1 }.mkString(",")
scala> sql(s"insert into test_$numCols values ($insertVals,3)")
scala> sql(s"select grouping_id(), sum(v) from test_$numCols group by grouping sets ((${cols.mkString(",")}), (${cols.init.mkString(",")}))").show(10, false)
scala> sql(s"drop table test_$numCols")

// numCols = 32
+-------------+------+
|grouping_id()|sum(v)|
+-------------+------+
|0            |3     |
|0            |3     | // Wrong Grouping ID
+-------------+------+

// numCols = 31
+-------------+------+
|grouping_id()|sum(v)|
+-------------+------+
|0            |3     |
|1            |3     |
+-------------+------+
```
To fix this issue, this pr change code to use long values for `GROUPING_ID` instead of int values.
### Why are the changes needed?

To support more cases in `GROUPING_ID`.

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

No.

### How was this patch tested?

Added unit tests.

Closes #26918 from maropu/FixGroupingIdIssue.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-03-06 16:57:03 +09:00
Burak Yavuz 2e3adadc6a [SPARK-31061][SQL] Provide ability to alter the provider of a table
This PR adds functionality to HiveExternalCatalog to be able to change the provider of a table.

This is useful for catalogs in Spark 3.0 to be able to use alterTable to change the provider of a table as part of an atomic REPLACE TABLE function.

No

Unit tests

Closes #27822 from brkyvz/externalCat.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-05 23:42:07 -08:00
Gengliang Wang 1426ad8968 [SPARK-23817][FOLLOWUP][TEST] Add OrcV2QuerySuite
### What changes were proposed in this pull request?

Add `OrcV2QuerySuite` which explicitly sets the configuration `USE_V1_SOURCE_LIST` as `""` to use ORC V2 implementation.

### Why are the changes needed?

As now file source V2 is disabled by default, the test suite `OrcQuerySuite` is testing V1 implementation as well as the `OrcV1QuerySuite`. We should fix it.
### Does this PR introduce any user-facing change?

No
### How was this patch tested?

Unit test.

Closes #27816 from gengliangwang/orcQuerySuite.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-03-05 21:22:32 -08:00
yi.wu 587266f887 [SPARK-31010][SQL][FOLLOW-UP] Deprecate untyped scala UDF
### What changes were proposed in this pull request?

Use scala annotation deprecate to deprecate untyped scala UDF.

### Why are the changes needed?

After #27488, it's weird to see the untyped scala UDF will fail by default without deprecation.

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

Yes, user will see the warning:
```
<console>:26: warning: method udf in object functions is deprecated (since 3.0.0): Untyped Scala UDF API is deprecated, please use typed Scala UDF API such as 'def udf[RT: TypeTag](f: Function0[RT]): UserDefinedFunction' instead.
       val myudf = udf(() => Math.random(), DoubleType)
                   ^
```

### How was this patch tested?

Tested manually.

Closes #27794 from Ngone51/deprecate_untyped_scala_udf.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-06 13:00:04 +08:00
Maxim Gekk 59f1e76b82 [SPARK-31020][SPARK-31023][SPARK-31025][SPARK-31044][SQL] Support foldable args by from_csv/json and schema_of_csv/json
### What changes were proposed in this pull request?
In the PR, I propose:

1. To replace matching by `Literal` in `ExprUtils.evalSchemaExpr()` to checking foldable property of the `schema` expression.
2. To replace matching by `Literal` in `ExprUtils.evalTypeExpr()` to checking foldable property of the `schema` expression.
3. To change checking of the input parameter in the `SchemaOfCsv` expression, and allow foldable `child` expression.
4. To change checking of the input parameter in the `SchemaOfJson` expression, and allow foldable `child` expression.

### Why are the changes needed?
This should improve Spark SQL UX for `from_csv`/`from_json`. Currently, Spark expects only literals:
```sql
spark-sql> select from_csv('1,Moscow', replace('dpt_org_id INT, dpt_org_city STRING', 'dpt_org_', ''));
Error in query: Schema should be specified in DDL format as a string literal or output of the schema_of_csv function instead of replace('dpt_org_id INT, dpt_org_city STRING', 'dpt_org_', '');; line 1 pos 7
spark-sql> select from_json('{"id":1, "city":"Moscow"}', replace('dpt_org_id INT, dpt_org_city STRING', 'dpt_org_', ''));
Error in query: Schema should be specified in DDL format as a string literal or output of the schema_of_json function instead of replace('dpt_org_id INT, dpt_org_city STRING', 'dpt_org_', '');; line 1 pos 7
```
and only string literals are acceptable as CSV examples by `schema_of_csv`/`schema_of_json`:
```sql
spark-sql> select schema_of_csv(concat_ws(',', 0.1, 1));
Error in query: cannot resolve 'schema_of_csv(concat_ws(',', CAST(0.1BD AS STRING), CAST(1 AS STRING)))' due to data type mismatch: The input csv should be a string literal and not null; however, got concat_ws(',', CAST(0.1BD AS STRING), CAST(1 AS STRING)).; line 1 pos 7;
'Project [unresolvedalias(schema_of_csv(concat_ws(,, cast(0.1 as string), cast(1 as string))), None)]
+- OneRowRelation
spark-sql> select schema_of_json(regexp_replace('{"item_id": 1, "item_price": 0.1}', 'item_', ''));
Error in query: cannot resolve 'schema_of_json(regexp_replace('{"item_id": 1, "item_price": 0.1}', 'item_', ''))' due to data type mismatch: The input json should be a string literal and not null; however, got regexp_replace('{"item_id": 1, "item_price": 0.1}', 'item_', '').; line 1 pos 7;
'Project [unresolvedalias(schema_of_json(regexp_replace({"item_id": 1, "item_price": 0.1}, item_, )), None)]
+- OneRowRelation
```

### Does this PR introduce any user-facing change?
Yes, after the changes users can pass any foldable string expression as the `schema` parameter to `from_csv()/from_json()`. For the example above:
```sql
spark-sql> select from_csv('1,Moscow', replace('dpt_org_id INT, dpt_org_city STRING', 'dpt_org_', ''));
{"id":1,"city":"Moscow"}
spark-sql> select from_json('{"id":1, "city":"Moscow"}', replace('dpt_org_id INT, dpt_org_city STRING', 'dpt_org_', ''));
{"id":1,"city":"Moscow"}
```
After change the `schema_of_csv`/`schema_of_json` functions accept foldable expressions, for example:
```sql
spark-sql> select schema_of_csv(concat_ws(',', 0.1, 1));
struct<_c0:double,_c1:int>
spark-sql> select schema_of_json(regexp_replace('{"item_id": 1, "item_price": 0.1}', 'item_', ''));
struct<id:bigint,price:double>
```

### How was this patch tested?
Added new test to `CsvFunctionsSuite` and to `JsonFunctionsSuite`.

Closes #27804 from MaxGekk/foldable-arg-csv-json-func.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-06 12:29:35 +08:00
Dongjoon Hyun afb84e9d37 [SPARK-30886][SQL] Deprecate two-parameter TRIM/LTRIM/RTRIM functions
### What changes were proposed in this pull request?

This PR aims to show a deprecation warning on two-parameter TRIM/LTRIM/RTRIM function usages based on the community decision.
- https://lists.apache.org/thread.html/r48b6c2596ab06206b7b7fd4bbafd4099dccd4e2cf9801aaa9034c418%40%3Cdev.spark.apache.org%3E

### Why are the changes needed?

For backward compatibility, SPARK-28093 is reverted. However, from Apache Spark 3.0.0, we should give a safe guideline to use SQL syntax instead of the esoteric function signatures.

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

Yes. This shows a directional warning.

### How was this patch tested?

Pass the Jenkins with a newly added test case.

Closes #27643 from dongjoon-hyun/SPARK-30886.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-03-05 20:09:39 -08:00
maryannxue d705d36c0c [SPARK-31045][SQL] Add config for AQE logging level
### What changes were proposed in this pull request?
This PR adds an internal config for changing the logging level of adaptive execution query plan evolvement.

### Why are the changes needed?
To make AQE debugging easier.

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

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

Closes #27798 from maryannxue/aqe-log-level.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-06 11:41:45 +08:00
beliefer d9254b26f1 [SPARK-30841][SQL][DOC][FOLLOW-UP] Add version information to the configuration of SQL
### What changes were proposed in this pull request?
This PR follows https://github.com/apache/spark/pull/27691 and https://github.com/apache/spark/pull/27730
I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.sql.execution.useObjectHashAggregateExec | 2.2.0 | SPARK-19944 | 0ee38a39e43dd7ad9d50457e446ae36f64621a1b#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.jsonGenerator.ignoreNullFields | 3.0.0 | SPARK-29444 | 78b0cbe265c4e8cc3d4d8bf5d734f2998c04d376#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.streaming.fileSink.log.deletion | 2.0.0 | SPARK-14678 | 7bc948557bb6169cbeec335f8400af09375a62d3#diff-32bb9518401c0948c5ea19377b5069ab |
spark.sql.streaming.fileSink.log.compactInterval | 2.0.0 | SPARK-14678 | 7bc948557bb6169cbeec335f8400af09375a62d3#diff-32bb9518401c0948c5ea19377b5069ab |
spark.sql.streaming.fileSink.log.cleanupDelay | 2.0.0 | SPARK-14678 | 7bc948557bb6169cbeec335f8400af09375a62d3#diff-32bb9518401c0948c5ea19377b5069ab |
spark.sql.streaming.fileSource.log.deletion | 2.0.1 | SPARK-15698 | 8d8e2332ca12067817de45a8d3812928150975d0#diff-32bb9518401c0948c5ea19377b5069ab |
spark.sql.streaming.fileSource.log.compactInterval | 2.0.1 | SPARK-15698 | 8d8e2332ca12067817de45a8d3812928150975d0#diff-32bb9518401c0948c5ea19377b5069ab |
spark.sql.streaming.fileSource.log.cleanupDelay | 2.0.1 | SPARK-15698 | 8d8e2332ca12067817de45a8d3812928150975d0#diff-32bb9518401c0948c5ea19377b5069ab |
spark.sql.streaming.fileSource.schema.forceNullable | 3.0.0 | SPARK-28651 | 5bb69945e4aaf519cd10a5c5083332f618039af0#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.streaming.fileSource.cleaner.numThreads | 3.0.0 | SPARK-29876 | abf759a91e01497586b8bb6b7a314dd28fd6cff1#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.streaming.schemaInference | 2.0.0 | SPARK-15458 | 1fb7b3a0a2e3a5c5f784aab662df93fcc1449c36#diff-32bb9518401c0948c5ea19377b5069ab |  
spark.sql.streaming.pollingDelay | 2.0.0 | SPARK-16002 | afa14b71b28d788c53816bd2616ccff0c3967f40#diff-32bb9518401c0948c5ea19377b5069ab |  
spark.sql.streaming.stopTimeout | 3.0.0 | SPARK-30143 | 4c37a8a3f4a489b52f1919d2db84f6e32c6a05cd#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.streaming.noDataProgressEventInterval | 2.1.1 | SPARK-19944 | 80ebca62cbdb7d5c8606e95a944164ab1a943694#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.streaming.noDataMicroBatches.enabled | 2.4.1 | SPARK-24157 | 535bf1cc9e6b54df7059ac3109b8cba30057d040#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.streaming.metricsEnabled | 2.0.2 | SPARK-17731 | 881e0eb05782ea74cf92a62954466b14ea9e05b6#diff-32bb9518401c0948c5ea19377b5069ab |
spark.sql.streaming.numRecentProgressUpdates | 2.1.1 | SPARK-19944 | 80ebca62cbdb7d5c8606e95a944164ab1a943694#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.streaming.checkpointFileManagerClass | 2.4.0 | SPARK-23966 | cbb41a0c5b01579c85f06ef42cc0585fbef216c5#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.streaming.checkpoint.escapedPathCheck.enabled | 3.0.0 | SPARK-26824 | 77b99af57330cf2e5016a6acc69642d54041b041#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.statistics.parallelFileListingInStatsComputation.enabled | 2.4.1 | SPARK-24626 | f11f44548903bbab7ab764574d6bed326cf4cd8d#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.defaultSizeInBytes | 1.1.0 | SPARK-2393 | c7db274be79f448fda566208946cb50958ea9b1a#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.statistics.fallBackToHdfs | 2.0.0 | SPARK-15960 | 5c53442cc098dd618ba1430962727c74b2de2e68#diff-32bb9518401c0948c5ea19377b5069ab |  
spark.sql.statistics.ndv.maxError | 2.1.1 | SPARK-19944 | 80ebca62cbdb7d5c8606e95a944164ab1a943694#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.statistics.histogram.enabled | 2.3.0 | SPARK-17074 | 11b60af737a04d931356aa74ebf3c6cf4a6b08d6#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.statistics.histogram.numBins | 2.3.0 | SPARK-17074 | 11b60af737a04d931356aa74ebf3c6cf4a6b08d6#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.statistics.percentile.accuracy | 2.3.0 | SPARK-17074 | 11b60af737a04d931356aa74ebf3c6cf4a6b08d6#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.statistics.size.autoUpdate.enabled | 2.3.0 | SPARK-21127 | d5202259d9aa9ad95d572af253bf4a722b7b437a#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.cbo.enabled | 2.2.0 | SPARK-19944 | 0ee38a39e43dd7ad9d50457e446ae36f64621a1b#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.cbo.planStats.enabled | 3.0.0 | SPARK-24690 | 3f3a18fff116a02ff7996d45a1061f48a2de3102#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.cbo.joinReorder.enabled | 2.2.0 | SPARK-19944 | 0ee38a39e43dd7ad9d50457e446ae36f64621a1b#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.cbo.joinReorder.dp.threshold | 2.2.0 | SPARK-19944 | 0ee38a39e43dd7ad9d50457e446ae36f64621a1b#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.cbo.joinReorder.card.weight | 2.2.0 | SPARK-19915 | c083b6b7dec337d680b54dabeaa40e7a0f69ae69#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.cbo.joinReorder.dp.star.filter | 2.2.0 | SPARK-20233 | fbe4216e1e83d243a7f0521b76bfb20c25278281#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.cbo.starSchemaDetection | 2.2.0 | SPARK-17791 | 81639115947a13017d1637549a8f66ba599b27b8#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.cbo.starJoinFTRatio | 2.2.0 | SPARK-17791 | 81639115947a13017d1637549a8f66ba599b27b8#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.session.timeZone | 2.2.0 | SPARK-19944 | 0ee38a39e43dd7ad9d50457e446ae36f64621a1b#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.windowExec.buffer.in.memory.threshold | 2.2.1 | SPARK-21595 | 406eb1c2ee670c2f14f2737c32c9aa0b8d35bf7c#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.windowExec.buffer.spill.threshold | 2.2.0 | SPARK-13450 | 02c274eaba0a8e7611226e0d4e93d3c36253f4ce#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.sortMergeJoinExec.buffer.in.memory.threshold | 2.2.1 | SPARK-21595 | 406eb1c2ee670c2f14f2737c32c9aa0b8d35bf7c#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.sortMergeJoinExec.buffer.spill.threshold | 2.2.0 | SPARK-13450 | 02c274eaba0a8e7611226e0d4e93d3c36253f4ce#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.cartesianProductExec.buffer.in.memory.threshold | 2.2.1 | SPARK-21595 | 406eb1c2ee670c2f14f2737c32c9aa0b8d35bf7c#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.cartesianProductExec.buffer.spill.threshold | 2.2.0 | SPARK-13450 | 02c274eaba0a8e7611226e0d4e93d3c36253f4ce#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parser.quotedRegexColumnNames | 2.3.0 | SPARK-12139 | 2cbfc975ba937a4eb761de7a6473b7747941f386#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.execution.rangeExchange.sampleSizePerPartition | 2.3.0 | SPARK-22160 | 323806e68f91f3c7521327186a37ddd1436267d0#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.execution.arrow.enabled | 2.3.0 | SPARK-22159 | d29d1e87995e02cb57ba3026c945c3cd66bb06e2#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.execution.arrow.pyspark.enabled | 3.0.0 | SPARK-27834 | db48da87f02e2e89710ba65fab8b07e9c85b9e74#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.execution.arrow.sparkr.enabled | 3.0.0 | SPARK-27834 | db48da87f02e2e89710ba65fab8b07e9c85b9e74#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.execution.arrow.fallback.enabled | 2.4.0 | SPARK-23380 | d6632d185e147fcbe6724545488ad80dce20277e#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.execution.arrow.pyspark.fallback.enabled | 3.0.0 | SPARK-27834 | db48da87f02e2e89710ba65fab8b07e9c85b9e74#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.execution.arrow.maxRecordsPerBatch | 2.3.0 | SPARK-13534 | d03aebbe6508ba441dc87f9546f27aeb27553d77#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.execution.pandas.udf.buffer.size | 3.1.0 | SPARK-27870 | 692e3ddb4e517638156f7427ade8b62fb37634a7#diff-9a6b543db706f1a90f790783d6930a13 | Exists in master, not branch-3.0
spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName | 2.4.1 | SPARK-24324 | 3f203050ac764516e68fb43628bba0df5963e44d#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.execution.pandas.convertToArrowArraySafely | 3.0.0 | SPARK-30812 | b76bc0b1b8b2abd00a84f805af90ca4c5925faaa#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.replaceExceptWithFilter | 2.3.0 | SPARK-22181 | 01f6ba0e7a12ef818d56e7d5b1bd889b79f2b57c#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.decimalOperations.allowPrecisionLoss | 2.3.1 | SPARK-22036 | 8a98274823a4671cee85081dd19f40146e736325#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.legacy.literal.pickMinimumPrecision | 2.3.3 | SPARK-25454 | 26d893a4f64de18222942568f7735114447a6ab7#diff-9a6b543db706f1a90f790783d6930a13 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27770 from beliefer/add-version-to-sql-config-part-three.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-06 11:19:08 +09:00
Maxim Gekk cf7c397ede [MINOR][SQL] Remove an ignored test from JsonSuite
### What changes were proposed in this pull request?
Remove ignored and outdated test `Type conflict in primitive field values (Ignored)` from JsonSuite.

### Why are the changes needed?
The test is not maintained for long time. It can be removed to reduce size of JsonSuite, and improve maintainability.

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

### How was this patch tested?
By running the command `./build/sbt "test:testOnly *JsonV2Suite"`

Closes #27795 from MaxGekk/remove-ignored-test-in-JsonSuite.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-06 10:35:44 +09:00
HyukjinKwon fc12165f48 [SPARK-31036][SQL] Use stringArgs in Expression.toString to respect hidden parameters
### What changes were proposed in this pull request?

This PR proposes to respect hidden parameters by using `stringArgs`  in `Expression.toString `. By this, we can show the strings properly in some cases such as `NonSQLExpression`.

### Why are the changes needed?

To respect "hidden" arguments in the string representation.

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

Yes, for example, on the top of https://github.com/apache/spark/pull/27657,

```scala
val identify = udf((input: Seq[Int]) => input)
spark.range(10).select(identify(array("id"))).show()
```

shows hidden parameter `useStringTypeWhenEmpty`.

```
+---------------------+
|UDF(array(id, false))|
+---------------------+
|                  [0]|
|                  [1]|
...
```

whereas:

```scala
spark.range(10).select(array("id")).show()
```

```
+---------+
|array(id)|
+---------+
|      [0]|
|      [1]|
...
```

### How was this patch tested?

Manually tested as below:

```scala
val identify = udf((input: Boolean) => input)
spark.range(10).select(identify(exists(array(col("id")), _ % 2 === 0))).show()
```

Before:

```
+-------------------------------------------------------------------------------------+
|UDF(exists(array(id), lambdafunction(((lambda 'x % 2) = 0), lambda 'x, false), true))|
+-------------------------------------------------------------------------------------+
|                                                                                 true|
|                                                                                false|
|                                                                                 true|
...
```

After:

```
+-------------------------------------------------------------------------------+
|UDF(exists(array(id), lambdafunction(((lambda 'x % 2) = 0), lambda 'x, false)))|
+-------------------------------------------------------------------------------+
|                                                                           true|
|                                                                          false|
|                                                                           true|
...
```

Closes #27788 from HyukjinKwon/arguments-str-repr.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-06 10:33:20 +09:00
Peter Toth 72b52a3cdf [SPARK-30563][SQL] Disable using commit coordinator with NoopDataSource
### What changes were proposed in this pull request?
This PR disables using commit coordinator with `NoopDataSource`.

### Why are the changes needed?
No need for a coordinator in benchmarks.

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

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

Closes #27791 from peter-toth/SPARK-30563-disalbe-commit-coordinator.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-06 10:30:59 +09:00
DB Tsai fe126a6a05 [SPARK-31058][SQL][TEST-HIVE1.2] Consolidate the implementation of quoteIfNeeded
### What changes were proposed in this pull request?
There are two implementation of quoteIfNeeded.  One is in `org.apache.spark.sql.connector.catalog.CatalogV2Implicits.quote` and the other is in `OrcFiltersBase.quoteAttributeNameIfNeeded`. This PR will consolidate them into one.

### Why are the changes needed?
Simplify the codebase.

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

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

Closes #27814 from dbtsai/SPARK-31058.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-03-06 00:13:57 +00:00
Wenchen Fan ba86524b25 [SPARK-31037][SQL] refine AQE config names
### What changes were proposed in this pull request?

When introducing AQE to others, I feel the config names are a bit incoherent and hard to use.
This PR refines the config names:
1. remove the "shuffle" prefix. AQE is all about shuffle and we don't need to add the "shuffle" prefix everywhere.
2. `targetPostShuffleInputSize` is obscure, rename to `advisoryShufflePartitionSizeInBytes`.
3. `reducePostShufflePartitions` doesn't match the actual optimization, rename to `coalesceShufflePartitions`
4. `minNumPostShufflePartitions` is obscure, rename it `minPartitionNum` under the `coalesceShufflePartitions` namespace
5. `maxNumPostShufflePartitions` is confusing with the word "max", rename it `initialPartitionNum`
6. `skewedJoinOptimization` is too verbose. skew join is a well-known terminology in database area, we can just say `skewJoin`

### Why are the changes needed?

Make the config names easy to understand.

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

deprecate the config `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`

### How was this patch tested?

N/A

Closes #27793 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-06 00:46:34 +08:00
Maxim Gekk 1fd9a91c66 [SPARK-31005][SQL] Support time zone ids in casting strings to timestamps
### What changes were proposed in this pull request?
In the PR, I propose to change `DateTimeUtils.stringToTimestamp` to support any valid time zone id at the end of input string. After the changes, the function accepts zone ids in the formats:
- no zone id. In that case, the function uses the local session time zone from the SQL config `spark.sql.session.timeZone`
- -[h]h:[m]m
- +[h]h:[m]m
- Z
- Short zone id, see https://docs.oracle.com/javase/8/docs/api/java/time/ZoneId.html#SHORT_IDS
- Zone ID starts with 'UTC+', 'UTC-', 'GMT+', 'GMT-', 'UT+' or 'UT-'. The ID is split in two, with a two or three letter prefix and a suffix starting with the sign. The suffix must be in the formats:
  - +|-h[h]
  - +|-hh[:]mm
  - +|-hh:mm:ss
  - +|-hhmmss
- Region-based zone IDs in the form `{area}/{city}`, such as `Europe/Paris` or `America/New_York`. The default set of region ids is supplied by the IANA Time Zone Database (TZDB).

### Why are the changes needed?
- To use `stringToTimestamp` as a substitution of removed `stringToTime`, see https://github.com/apache/spark/pull/27710#discussion_r385020173
- Improve UX of Spark SQL by allowing flexible formats of zone ids. Currently, Spark accepts only `Z` and zone offsets that can be inconvenient when a time zone offset is shifted due to daylight saving rules. For instance:
```sql
spark-sql> select cast('2015-03-18T12:03:17.123456 Europe/Moscow' as timestamp);
NULL
```

### Does this PR introduce any user-facing change?
Yes. After the changes, casting strings to timestamps allows time zone id at the end of the strings:
```sql
spark-sql> select cast('2015-03-18T12:03:17.123456 Europe/Moscow' as timestamp);
2015-03-18 12:03:17.123456
```

### How was this patch tested?
- Added new test cases to the `string to timestamp` test in `DateTimeUtilsSuite`.
- Run `CastSuite` and `AnsiCastSuite`.

Closes #27753 from MaxGekk/stringToTimestamp-uni-zoneId.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-05 20:49:43 +08:00
Wenchen Fan 807ea413b4 [SPARK-31019][SQL] make it clear that people can deduplicate map keys
### What changes were proposed in this pull request?

rename the config and make it non-internal.

### Why are the changes needed?

Now we fail the query if duplicated map keys are detected, and provide a legacy config to deduplicate it. However, we must provide a way to get users out of this situation, instead of just rejecting to run the query. This exit strategy should always be there, while legacy config indicates that it may be removed someday.

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

no, just rename a config which was added in 3.0

### How was this patch tested?

add more tests for the fail behavior.

Closes #27772 from cloud-fan/map.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 20:43:52 +09:00
Kent Yao f45ae7f2c5 [SPARK-31038][SQL] Add checkValue for spark.sql.session.timeZone
### What changes were proposed in this pull request?

The `spark.sql.session.timeZone` config can accept any string value including invalid time zone ids, then it will fail other queries that rely on the time zone. We should do the value checking in the set phase and fail fast if the zone value is invalid.

### Why are the changes needed?

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

yes, will fail fast if the value is a wrong timezone id
### How was this patch tested?

add ut

Closes #27792 from yaooqinn/SPARK-31038.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-05 19:38:20 +08:00
maryannxue 9b602e26d2 [SPARK-31046][SQL] Make more efficient and clean up AQE update UI code
### What changes were proposed in this pull request?
This PR avoids sending redundant metrics (those that have been included in previous update) as well as useless metrics (those in future stages) to Spark UI in AQE UI metrics update.

### Why are the changes needed?
This change will make UI metrics update more efficient.

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

### How was this patch tested?
Manual test in Spark UI.

Closes #27799 from maryannxue/aqe-ui-cleanup.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-05 18:53:01 +08:00
Terry Kim 66b4fd040e [SPARK-31024][SQL] Allow specifying session catalog name spark_catalog in qualified column names for v1 tables
### What changes were proposed in this pull request?

Currently, the user cannot specify the session catalog name (`spark_catalog`) in qualified column names for v1 tables:
```
SELECT spark_catalog.default.t.i FROM spark_catalog.default.t
```
fails with `cannot resolve 'spark_catalog.default.t.i`.

This is inconsistent with v2 table behavior where catalog name can be used:
```
SELECT testcat.ns1.tbl.id FROM testcat.ns1.tbl.id
```

This PR proposes to fix the inconsistency and allow the user to specify session catalog name in column names for v1 tables.

### Why are the changes needed?

Fixing an inconsistent behavior.

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

Yes, now the following query works:
```
SELECT spark_catalog.default.t.i FROM spark_catalog.default.t
```

### How was this patch tested?

Added new tests.

Closes #27776 from imback82/spark_catalog_col_name_resolution.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-05 18:33:59 +08:00
Yuanjian Li 7db0af5785 [SPARK-30668][SQL][FOLLOWUP] Raise exception instead of silent change for new DateFormatter
### What changes were proposed in this pull request?
This is a follow-up work for #27441. For the cases of new TimestampFormatter return null while legacy formatter can return a value, we need to throw an exception instead of silent change. The legacy config will be referenced in the error message.

### Why are the changes needed?
Avoid silent result change for new behavior in 3.0.

### Does this PR introduce any user-facing change?
Yes, an exception is thrown when we detect legacy formatter can parse the string and the new formatter return null.

### How was this patch tested?
Extend existing UT.

Closes #27537 from xuanyuanking/SPARK-30668-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-05 15:29:39 +08:00
DB Tsai 3c16fae5c1 [SPARK-31027][SQL] Refactor DataSourceStrategy to be more extendable
### What changes were proposed in this pull request?
Refactor `DataSourceStrategy.scala` and `DataSourceStrategySuite.scala` so it's more extendable to implement nested predicate pushdown.

### Why are the changes needed?
To support nested predicate pushdown.

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

### How was this patch tested?
Existing tests and new tests.

Closes #27778 from dbtsai/SPARK-31027.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-04 23:41:49 +09:00
Terry Kim b30278107f [SPARK-30885][SQL][FOLLOW-UP] Fix issues where some V1 commands allow tables that are not fully qualified
### What changes were proposed in this pull request?

There are few V1 commands such as `REFRESH TABLE` that still allow `spark_catalog.t` because they run the commands with parsed table names without trying to load them in the catalog. This PR addresses this issue.

The PR also addresses the issue brought up in https://github.com/apache/spark/pull/27642#discussion_r382402104.

### Why are the changes needed?

To fix a bug where for some V1 commands, `spark_catalog.t` is allowed.

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

Yes, a bug is fixed and `REFRESH TABLE spark_catalog.t` is not allowed.

### How was this patch tested?

Added new test.

Closes #27718 from imback82/fix_TempViewOrV1Table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-04 18:09:48 +08:00
Wenchen Fan e4c61e35da [SPARK-30960][SQL] add back the legacy date/timestamp format support in CSV/JSON parser
### What changes were proposed in this pull request?

Before Spark 3.0, the JSON/CSV parser has a special behavior that, when the parser fails to parse a timestamp/date, fallback to another way to parse it, to support some legacy format. The fallback was removed by https://issues.apache.org/jira/browse/SPARK-26178 and https://issues.apache.org/jira/browse/SPARK-26243.

This PR adds back this legacy fallback. Since we switch the API to do datetime operations, we can't be exactly the same as before. Here we add back the support of the legacy formats that are common (examples of Spark 2.4):
1. the fields can have one or two letters
```
scala> sql("""select from_json('{"time":"1123-2-22 2:22:22"}', 'time Timestamp')""").show(false)
+-------------------------------------------+
|jsontostructs({"time":"1123-2-22 2:22:22"})|
+-------------------------------------------+
|[1123-02-22 02:22:22]                      |
+-------------------------------------------+
```
2. the separator between data and time can be "T" as well
```
scala> sql("""select from_json('{"time":"2000-12-12T12:12:12"}', 'time Timestamp')""").show(false)
+---------------------------------------------+
|jsontostructs({"time":"2000-12-12T12:12:12"})|
+---------------------------------------------+
|[2000-12-12 12:12:12]                        |
+---------------------------------------------+
```
3. the second fraction can be arbitrary length
```
scala> sql("""select from_json('{"time":"1123-02-22T02:22:22.123456789123"}', 'time Timestamp')""").show(false)
+----------------------------------------------------------+
|jsontostructs({"time":"1123-02-22T02:22:22.123456789123"})|
+----------------------------------------------------------+
|[1123-02-15 02:22:22.123]                                 |
+----------------------------------------------------------+
```
4. date string can end up with any chars after "T" or space
```
scala> sql("""select from_json('{"time":"1123-02-22Tabc"}', 'time date')""").show(false)
+----------------------------------------+
|jsontostructs({"time":"1123-02-22Tabc"})|
+----------------------------------------+
|[1123-02-22]                            |
+----------------------------------------+
```
5. remove "GMT" from the string before parsing
```
scala> sql("""select from_json('{"time":"GMT1123-2-22 2:22:22.123"}', 'time Timestamp')""").show(false)
+--------------------------------------------------+
|jsontostructs({"time":"GMT1123-2-22 2:22:22.123"})|
+--------------------------------------------------+
|[1123-02-22 02:22:22.123]                         |
+--------------------------------------------------+
```
### Why are the changes needed?

It doesn't hurt to keep this legacy support. It just makes the parsing more relaxed.

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

yes, to make 3.0 support parsing most of the csv/json values that were supported before.

### How was this patch tested?

new tests

Closes #27710 from cloud-fan/bug2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-04 18:27:44 +09:00
Shixiong Zhu ebfff7af6a [SPARK-30984][SS] Add UI test for Structured Streaming UI
### What changes were proposed in this pull request?

- Add a UI test for Structured Streaming UI
- Fix the unsafe usages of `SimpleDateFormat` by using a ThreadLocal shared object.
- Use `start` to replace `submission` to be consistent with the API `StreamingQuery.start()`.

### Why are the changes needed?

Structured Streaming UI is missing UI tests.

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

No

### How was this patch tested?

The new test.

Closes #27732 from zsxwing/ss-ui-test.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-04 13:55:34 +08:00
yi.wu 380e887631 [SPARK-30999][SQL] Don't cancel a QueryStageExec which failed before call doMaterialize
### What changes were proposed in this pull request?

This PR proposes to not cancel a `QueryStageExec` which failed before calling `doMaterialize`.

Besides, this PR also includes 2 minor improvements:

* fail fast when stage failed before calling `doMaterialize`

* format Exception with Cause

### Why are the changes needed?

For a stage which failed before materializing the lazy value (e.g. `inputRDD`), calling `cancel` on it could re-trigger the same failure again, e.g. executing child node again(see `AdaptiveQueryExecSuite`.`SPARK-30291: AQE should catch the exceptions when doing materialize` for example). And finally, the same failure will be counted 2 times, one is for materialize error and another is for cancel error.

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

No.

### How was this patch tested?

Updated test.

Closes #27752 from Ngone51/avoid_cancel_finished_stage.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2020-03-03 13:40:51 -08:00
Takeshi Yamamuro 4a1d273a4a [SPARK-30997][SQL] Fix an analysis failure in generators with aggregate functions
### What changes were proposed in this pull request?

We have supported generators in SQL aggregate expressions by SPARK-28782.
But, the generator(explode) query with aggregate functions in DataFrame failed as follows;

```
// SPARK-28782: Generator support in aggregate expressions
scala> spark.range(3).toDF("id").createOrReplaceTempView("t")
scala> sql("select explode(array(min(id), max(id))) from t").show()
+---+
|col|
+---+
|  0|
|  2|
+---+

// A failure case handled in this pr
scala> spark.range(3).select(explode(array(min($"id"), max($"id")))).show()
org.apache.spark.sql.AnalysisException:
The query operator `Generate` contains one or more unsupported
expression types Aggregate, Window or Generate.
Invalid expressions: [min(`id`), max(`id`)];;
Project [col#46L]
+- Generate explode(array(min(id#42L), max(id#42L))), false, [col#46L]
   +- Range (0, 3, step=1, splits=Some(4))

  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis(CheckAnalysis.scala:49)
  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis$(CheckAnalysis.scala:48)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:129)
```

The root cause is that `ExtractGenerator` wrongly replaces a project w/ aggregate functions
before `GlobalAggregates` replaces it with an aggregate as follows;

```
scala> sql("SET spark.sql.optimizer.planChangeLog.level=warn")
scala> spark.range(3).select(explode(array(min($"id"), max($"id")))).show()

20/03/01 12:51:58 WARN HiveSessionStateBuilder$$anon$1:
=== Applying Rule org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences ===
!'Project [explode(array(min('id), max('id))) AS List()]   'Project [explode(array(min(id#72L), max(id#72L))) AS List()]
 +- Range (0, 3, step=1, splits=Some(4))                   +- Range (0, 3, step=1, splits=Some(4))

20/03/01 12:51:58 WARN HiveSessionStateBuilder$$anon$1:
=== Applying Rule org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator ===
!'Project [explode(array(min(id#72L), max(id#72L))) AS List()]   Project [col#76L]
!+- Range (0, 3, step=1, splits=Some(4))                         +- Generate explode(array(min(id#72L), max(id#72L))), false, [col#76L]
!                                                                   +- Range (0, 3, step=1, splits=Some(4))

20/03/01 12:51:58 WARN HiveSessionStateBuilder$$anon$1:
=== Result of Batch Resolution ===
!'Project [explode(array(min('id), max('id))) AS List()]   Project [col#76L]
!+- Range (0, 3, step=1, splits=Some(4))                   +- Generate explode(array(min(id#72L), max(id#72L))), false, [col#76L]
!                                                             +- Range (0, 3, step=1, splits=Some(4))

// the analysis failed here...
```

To avoid the case in `ExtractGenerator`, this pr addes a condition to ignore generators having aggregate functions.
A correct sequence of rules is as follows;

```
20/03/01 13:19:06 WARN HiveSessionStateBuilder$$anon$1:
=== Applying Rule org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences ===
!'Project [explode(array(min('id), max('id))) AS List()]   'Project [explode(array(min(id#27L), max(id#27L))) AS List()]
 +- Range (0, 3, step=1, splits=Some(4))                   +- Range (0, 3, step=1, splits=Some(4))

20/03/01 13:19:06 WARN HiveSessionStateBuilder$$anon$1:
=== Applying Rule org.apache.spark.sql.catalyst.analysis.Analyzer$GlobalAggregates ===
!'Project [explode(array(min(id#27L), max(id#27L))) AS List()]   'Aggregate [explode(array(min(id#27L), max(id#27L))) AS List()]
 +- Range (0, 3, step=1, splits=Some(4))                         +- Range (0, 3, step=1, splits=Some(4))

20/03/01 13:19:06 WARN HiveSessionStateBuilder$$anon$1:
=== Applying Rule org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator ===
!'Aggregate [explode(array(min(id#27L), max(id#27L))) AS List()]   'Project [explode(_gen_input_0#31) AS List()]
!+- Range (0, 3, step=1, splits=Some(4))                           +- Aggregate [array(min(id#27L), max(id#27L)) AS _gen_input_0#31]
!                                                                     +- Range (0, 3, step=1, splits=Some(4))

```

### Why are the changes needed?

A bug fix.

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

No.

### How was this patch tested?

Added tests.

Closes #27749 from maropu/ExplodeInAggregate.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-03-03 12:25:12 -08:00
Terry Kim c263c15408 [SPARK-31015][SQL] Star(*) expression fails when used with qualified column names for v2 tables
### What changes were proposed in this pull request?

For a v2 table created with `CREATE TABLE testcat.ns1.ns2.tbl (id bigint, name string) USING foo`, the following works as expected
```
SELECT testcat.ns1.ns2.tbl.id FROM testcat.ns1.ns2.tbl
```
, but a query with qualified column name with star(*)
```
SELECT testcat.ns1.ns2.tbl.* FROM testcat.ns1.ns2.tbl
[info]   org.apache.spark.sql.AnalysisException: cannot resolve 'testcat.ns1.ns2.tbl.*' given input columns 'id, name';
```
fails to resolve. And this PR proposes to fix this issue.

### Why are the changes needed?

To fix a bug as describe above.

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

Yes, now `SELECT testcat.ns1.ns2.tbl.* FROM testcat.ns1.ns2.tbl` works as expected.

### How was this patch tested?

Added new test.

Closes #27766 from imback82/fix_star_expression.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-04 00:55:26 +08:00
Javier 3ff2135686 [SPARK-30049][SQL] SQL fails to parse when comment contains an unmatched quote character
### What changes were proposed in this pull request?

A SQL statement that contains a comment with an unmatched quote character can lead to a parse error:
- Added a insideComment flag in the splitter method to avoid checking single and double quotes within a comment:
```
spark-sql> SELECT 1 -- someone's comment here
         > ;
Error in query:
extraneous input ';' expecting <EOF>(line 2, pos 0)

== SQL ==
SELECT 1 -- someone's comment here
;
^^^
```

### Why are the changes needed?

This misbehaviour was not present on previous spark versions.

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

- No

### How was this patch tested?

- New tests were added.

Closes #27321 from javierivanov/SPARK-30049B.

Lead-authored-by: Javier <jfuentes@hortonworks.com>
Co-authored-by: Javier Fuentes <j.fuentes.m@icloud.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-03-03 09:55:15 -06:00