Commit graph

6805 commits

Author SHA1 Message Date
Max Gekk f1fde0cc22 [SPARK-31490][SQL][TESTS] Benchmark conversions to/from Java 8 datetime types
### What changes were proposed in this pull request?
- Add benchmark cases for **parallelizing** `java.time.LocalDate` and `java.time.Instant` column values.
- Add benchmark cases for **collecting** `java.time.LocalDate` and `java.time.Instant` column values.

### Why are the changes needed?
- To detect perf regression in the future
- To compare parallelization/collection of Java 8 date-time types with Java 7 date-time types `java.sql.Date` & `java.sql.Timestamp`.

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

### How was this patch tested?
By running the modified benchmarks in the environment:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_242 and OpenJDK 64-Bit Server VM 11.0.6+10 |

Closes #28263 from MaxGekk/java8-datetime-collect-benchmark.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-20 07:26:38 +00:00
Terry Kim d7499aed9c [SPARK-31256][SQL] DataFrameNaFunctions.drop should work for nested columns
### What changes were proposed in this pull request?

#26700 removed the ability to drop a row whose nested column value is null.

For example, for the following `df`:
```
val schema = new StructType()
  .add("c1", new StructType()
    .add("c1-1", StringType)
    .add("c1-2", StringType))
val data = Seq(Row(Row(null, "a2")), Row(Row("b1", "b2")), Row(null))
val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
df.show
+--------+
|      c1|
+--------+
|  [, a2]|
|[b1, b2]|
|    null|
+--------+
```
In Spark 2.4.4,
```
df.na.drop("any", Seq("c1.c1-1")).show
+--------+
|      c1|
+--------+
|[b1, b2]|
+--------+
```
In Spark 2.4.5 or Spark 3.0.0-preview2, if nested columns are specified, they are ignored.
```
df.na.drop("any", Seq("c1.c1-1")).show
+--------+
|      c1|
+--------+
|  [, a2]|
|[b1, b2]|
|    null|
+--------+
```
### Why are the changes needed?

This seems like a regression.

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

Now, the nested column can be specified:
```
df.na.drop("any", Seq("c1.c1-1")).show
+--------+
|      c1|
+--------+
|[b1, b2]|
+--------+
```

Also, if `*` is specified as a column, it will throw an `AnalysisException` that `*` cannot be resolved, which was the behavior in 2.4.4. Currently, in master, it has no effect.

### How was this patch tested?

Updated existing tests.

Closes #28266 from imback82/SPARK-31256.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-20 02:59:09 +00:00
Takeshi Yamamuro 74aed8cc8b
[SPARK-31476][SQL] Add an ExpressionInfo entry for EXTRACT
### What changes were proposed in this pull request?

This PR intends to add an ExpressionInfo entry for EXTRACT for better documentations.
This PR comes from the comment in https://github.com/apache/spark/pull/21479#discussion_r409900080

### Why are the changes needed?

To make SQL documentations complete.

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

Yes, this PR updates the `Spark SQL, Built-in Functions` page.

### How was this patch tested?

Run the example tests.

Closes #28251 from maropu/AddExtractExpr.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-18 13:37:12 -07:00
ulysses 6c2bf8248a
[SPARK-31442][SQL] Print shuffle id at coalesce partitions target size
### What changes were proposed in this pull request?

Minor change. Print shuffle id.

### Why are the changes needed?

Make log more clear.

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

No.

### How was this patch tested?

Not need.

Closes #28211 from ulysses-you/print-shuffle-id.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-18 09:27:44 -07:00
gatorsmile 6bf5f01a4a [SPARK-31477][SQL] Dump codegen and compile time in BenchmarkQueryTest
### What changes were proposed in this pull request?
This PR is to dump the codegen and compilation time for benchmark query tests.

### Why are the changes needed?
Measure the codegen and compilation time costs in TPC-DS queries

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

### How was this patch tested?
Manual test in my local laptop:
```
23:13:12.845 WARN org.apache.spark.sql.TPCDSQuerySuite:
=== Metrics of Whole-stage Codegen ===
Total code generation time: 21.275102261 seconds
Total compilation time: 12.223771828 seconds
```

Closes #28252 from gatorsmile/testMastercode.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-04-18 20:59:45 +09:00
Kent Yao 77cb7cde0d
[SPARK-31469][SQL][TESTS][FOLLOWUP] Remove unsupported fields from ExtractBenchmark
### What changes were proposed in this pull request?

In 697083c051, we remove  "MILLENNIUM", "CENTURY", "DECADE",  "QUARTER", "MILLISECONDS", "MICROSECONDS", "EPOCH" field for date_part and extract expression, this PR fix the related Benchmark.
### Why are the changes needed?

test fix.

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

no
### How was this patch tested?

passing Jenkins

Closes #28249 from yaooqinn/SPARK-31469-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-18 00:32:42 -07:00
Maryann Xue 6198f38405
[SPARK-31473][SQL] AQE should set active session during execution
### What changes were proposed in this pull request?

AQE creates new SparkPlan nodes during execution. This PR makes sure that the active session is set correctly during this process and AQE execution is not disrupted by external session change.

### Why are the changes needed?

To prevent potential errors. If not changed, the physical plans generated by AQE would have the wrong SparkSession or even null SparkSession, which could lead to NPE.

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

No.

### How was this patch tested?

Added UT.

Closes #28247 from maryannxue/aqe-activesession.

Authored-by: Maryann Xue <maryann.xue@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-18 00:08:36 -07:00
Wenchen Fan db7b8651a1 [SPARK-31253][SQL][FOLLOW-UP] simplify the code of calculating size metrics of AQE shuffle
### What changes were proposed in this pull request?

A followup of https://github.com/apache/spark/pull/28175:
1. use mutable collection to store the driver metrics
2. don't send size metrics if there is no map stats, as UI will display size as 0 if there is no data
3. calculate partition data size separately, to make the code easier to read.

### Why are the changes needed?

code simplification

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

no

### How was this patch tested?

existing tests

Closes #28240 from cloud-fan/refactor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-04-17 13:20:34 -07:00
Kent Yao 697083c051 [SPARK-31469][SQL] Make extract interval field ANSI compliance
### What changes were proposed in this pull request?

Currently, we can extract `millennium/century/decade/year/quarter/month/week/day/hour/minute/second(with fractions)//millisecond/microseconds` and `epoch` from interval values

While getting the `millennium/century/decade/year`, it means how many the interval `months` part can be converted to that unit-value. The content of `millennium/century/decade` will overlap `year` and each other.

While getting `month/day` and so on, it means the integral remainder of the previous unit. Here all the units including `year` are individual.

So while extracting `year`, `month`, `day`, `hour`, `minute`, `second`, which are ANSI primary datetime units, the semantic is `extracting`, but others might refer to `transforming`.

While getting epoch we have treat month as 30 days which varies the natural Calendar rules we use.

To avoid ambiguity, I suggest we should only support those extract field defined ANSI with their abbreviations.

### Why are the changes needed?

Extracting `millennium`, `century` etc does not obey the meaning of extracting, and they are not so useful and worth maintaining.

The `extract` is ANSI standard expression and `date_part` is its pg-specific alias function. The current support extract-fields are fully bought from PostgreSQL.

With a look at other systems like Presto/Hive, they don't support those ambiguous fields too.

e.g. Hive 2.2.x also take it from PostgreSQL but without introducing those ambiguous fields https://issues.apache.org/jira/secure/attachment/12828349/HIVE-14579

e.g. presto

```sql
presto> select extract(quater from interval '10-0' year to month);
Query 20200417_094723_00020_m8xq4 failed: line 1:8: Invalid EXTRACT field: quater
select extract(quater from interval '10-0' year to month)

presto> select extract(decade from interval '10-0' year to month);
Query 20200417_094737_00021_m8xq4 failed: line 1:8: Invalid EXTRACT field: decade
select extract(decade from interval '10-0' year to month)

```

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

Yes, as we already have previews versions, this PR will remove support for extracting `millennium/century/decade/quarter/week/millisecond/microseconds` and `epoch` from intervals with `date_part` function

### How was this patch tested?

rm some used tests

Closes #28242 from yaooqinn/SPARK-31469.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-17 13:59:02 +00:00
jiake d136b7248e [SPARK-31253][SQL][FOLLOW-UP] Improve the partition data size metrics in CustomShuffleReaderExec
### What changes were proposed in this pull request?
Currently the partition data size metrics contain three entries (min/max/avg)  in Spark UI, which is not user friendly. This PR lets the metrics with min/max/avg in one entry by calling SQLMetrics.postDriverMetricUpdates multiple times.
Before this PR, the spark UI is shown in the following:
![image](https://user-images.githubusercontent.com/11972570/78980137-da1a2200-7b4f-11ea-81ee-76858e887bde.png)

After this PR. the spark UI is shown in the following:
![image](https://user-images.githubusercontent.com/11972570/78980192-fae27780-7b4f-11ea-9faa-07f58699acfd.png)

### Why are the changes needed?
Improving UI

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

### How was this patch tested?
existing ut

Closes #28175 from JkSelf/improveAqeMetrics.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-17 06:23:54 +00:00
yi.wu 40f9dbb628 [SPARK-31425][SQL][CORE] UnsafeKVExternalSorter/VariableLengthRowBasedKeyValueBatch should also respect UnsafeAlignedOffset
### What changes were proposed in this pull request?

Make `UnsafeKVExternalSorter` / `VariableLengthRowBasedKeyValueBatch ` also respect `UnsafeAlignedOffset` when reading the record and update some out of date comemnts.

### Why are the changes needed?

Since `BytesToBytesMap` respects `UnsafeAlignedOffset` when writing the record, `UnsafeKVExternalSorter` should also respect `UnsafeAlignedOffset` when reading the record from `BytesToBytesMap` otherwise it will causes data correctness issue.

Unlike `UnsafeKVExternalSorter` may reading records from `BytesToBytesMap`, `VariableLengthRowBasedKeyValueBatch` writes and reads records by itself. Thus, similar to #22053 and [comment](https://github.com/apache/spark/pull/22053#issuecomment-411975239) there, fix for `VariableLengthRowBasedKeyValueBatch` more likely an improvement for the support of SPARC platform.

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

No.

### How was this patch tested?

Manually tested `HashAggregationQueryWithControlledFallbackSuite` with `UAO_SIZE=8`  to simulate SPARC platform. And tests only pass with this fix.

Closes #28195 from Ngone51/fix_uao.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-17 04:48:27 +00:00
herman fab4ca5156
[SPARK-31450][SQL] Make ExpressionEncoder thread-safe
### What changes were proposed in this pull request?
This PR moves the `ExpressionEncoder.toRow` and `ExpressionEncoder.fromRow` functions into their own function objects(`ExpressionEncoder.Serializer` & `ExpressionEncoder.Deserializer`). This effectively makes the `ExpressionEncoder` stateless, thread-safe and (more) reusable. The function objects are not thread safe, however they are documented as such and should be used in a more limited scope (making it easier to reason about thread safety).

### Why are the changes needed?
ExpressionEncoders are not thread-safe. We had various (nasty) bugs because of this.

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

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

Closes #28223 from hvanhovell/SPARK-31450.

Authored-by: herman <herman@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-16 18:47:46 -07:00
Peter Toth 7ad6ba36f2 [SPARK-30564][SQL] Revert Block.length and use comment placeholders in HashAggregateExec
### What changes were proposed in this pull request?
SPARK-21870 (cb0cddf#diff-06dc5de6163687b7810aa76e7e152a76R146-R149) caused significant performance regression in cases where the source code size is fairly large as `HashAggregateExec` uses `Block.length` to decide on splitting the code. The change in `length` makes sense as the comment and extra new lines shouldn't be taken into account when deciding on splitting, but the regular expression based approach is very slow and adds a big relative overhead to cases where the execution is quick (small number of rows).
This PR:
- restores `Block.length` to its original form
- places comments in `HashAggragateExec` with `CodegenContext.registerComment` so as to appear only when comments are enabled (`spark.sql.codegen.comments=true`)

Before this PR:
```
deeply nested struct field r/w:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
250 deep x 400 rows (read in-mem)                  1137           1143           8          0.1       11368.3       0.0X
```

After this PR:
```
deeply nested struct field r/w:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
250 deep x 400 rows (read in-mem)                   167            180           7          0.6        1674.3       0.1X
```
### Why are the changes needed?
To fix performance regression.

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

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

Closes #28083 from peter-toth/SPARK-30564-use-comment-placeholders.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-04-16 17:52:22 +09:00
Max Gekk c76c31e2c6 [SPARK-31455][SQL] Fix rebasing of not-existed timestamps
### What changes were proposed in this pull request?
In the PR, I propose to change rebasing of not-existed timestamps in the hybrid calendar (Julian + Gregorian since 1582-10-15) in the range [1582-10-05, 1582-10-15). Not existed timestamps from the range are shifted to the first valid date in the hybrid calendar - 1582-10-15. The changes affect only `rebaseGregorianToJulianMicros()` because reverse rebasing from the hybrid timestamps to Proleptic Gregorian timestamps does not have such problem.

The shifting affects only the date part of timestamps while keeping the time part as is. For example:
```
1582-10-10 00:11:22.334455 -> 1582-10-15 00:11:22.334455
```

### Why are the changes needed?
Currently, not-existed timestamps are shifted by standard difference between Julian and Gregorian calendar on 1582-10-04, for example 1582-10-14 00:00:00 -> 1582-10-24 00:00:00. That contradicts to shifting of not existed dates in other cases, for example:
```
scala> sql("select timestamp'1990-9-31 12:12:12'").show
+----------------------------------+
|TIMESTAMP('1990-10-01 12:12:12.0')|
+----------------------------------+
|               1990-10-01 12:12:12|
+----------------------------------+
```

### Does this PR introduce any user-facing change?
Yes, this impacts on conversion of Spark SQL `TIMESTAMP` values to external timestamps based on non-Proleptic Gregorian calendar. For example, while saving the 1582-10-14 12:13:14 date to ORC files, it will be shifted to the next valid date 1582-10-15 12:13:14.

### How was this patch tested?
- Added tests to `RebaseDateTimeSuite` and to `OrcSourceSuite`
- By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite`, `DateExpressionsSuite`, `CollectionExpressionsSuite`, `ParquetIOSuite`.

Closes #28227 from MaxGekk/fix-not-exist-timestamps.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-16 02:54:38 +00:00
Max Gekk 2b10d70bad [SPARK-31423][SQL] Fix rebasing of not-existed dates
### What changes were proposed in this pull request?
In the PR, I propose to change rebasing of not-existed dates in the hybrid calendar (Julian + Gregorian since 1582-10-15) in the range (1582-10-04, 1582-10-15). Not existed dates from the range are shifted to the first valid date in the hybrid calendar - 1582-10-15. The changes affect only `rebaseGregorianToJulianDays()` because reverse rebasing from the hybrid dates to Proleptic Gregorian dates does not have such problem.

### Why are the changes needed?
Currently, not-existed dates are shifted by standard difference between Julian and Gregorian calendar on 1582-10-04, for example 1582-10-14 -> 1582-10-24. That's contradict to shifting not existed dates in other cases, for example:
```
scala> sql("select date'1990-9-31'").show
+-----------------+
|DATE '1990-10-01'|
+-----------------+
|       1990-10-01|
+-----------------+
```

### Does this PR introduce any user-facing change?
Yes, this impacts on conversion of Spark SQL `DATE` values to external dates based on non-Proleptic Gregorian calendar. For example, while saving the 1582-10-14 date to ORC files, it will be shifted to the next valid date 1582-10-15.

### How was this patch tested?
- Added tests to `RebaseDateTimeSuite` and to `OrcSourceSuite`
- By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite`, `DateExpressionsSuite`, `CollectionExpressionsSuite`, `ParquetIOSuite`.

Closes #28225 from MaxGekk/fix-not-exist-dates.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-15 16:33:56 +00:00
Max Gekk 744c2480b5 [SPARK-31443][SQL] Fix perf regression of toJavaDate
### What changes were proposed in this pull request?
Optimise the `toJavaDate()` method of `DateTimeUtils` by:
1. Re-using `rebaseGregorianToJulianDays` optimised by #28067
2. Creating `java.sql.Date` instances from milliseconds in UTC since the epoch instead of date-time fields. This allows to avoid "normalization" inside of  `java.sql.Date`.

Also new benchmark for collecting dates is added to `DateTimeBenchmark`.

### Why are the changes needed?
The changes fix the performance regression of collecting `DATE` values comparing to Spark 2.4 (see `DateTimeBenchmark` in https://github.com/MaxGekk/spark/pull/27):

Spark 2.4.6-SNAPSHOT:
```
To/from Java's date-time:                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Date                                  559            603          38          8.9         111.8       1.0X
Collect dates                                      2306           3221        1558          2.2         461.1       0.2X
```
Before the changes:
```
To/from Java's date-time:                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Date                                 1052           1130          73          4.8         210.3       1.0X
Collect dates                                      3251           4943        1624          1.5         650.2       0.3X
```
After:
```
To/from Java's date-time:                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Date                                  416            419           3         12.0          83.2       1.0X
Collect dates                                      1928           2759        1180          2.6         385.6       0.2X
```

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

### How was this patch tested?
- By existing tests suites, in particular, `DateTimeUtilsSuite`, `RebaseDateTimeSuite`, `DateFunctionsSuite`, `DateExpressionsSuite`.
- Re-run `DateTimeBenchmark` in the environment:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_242 and OpenJDK 64-Bit Server VM 11.0.6+10 |

Closes #28212 from MaxGekk/optimize-toJavaDate.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-15 06:19:12 +00:00
Max Gekk 2c5d489679 [SPARK-31439][SQL] Fix perf regression of fromJavaDate
### What changes were proposed in this pull request?
In the PR, I propose to re-use optimized implementation of days rebase function `rebaseJulianToGregorianDays()` introduced by the PR #28067 in conversion of `java.sql.Date` values to Catalyst's `DATE` values. The function `fromJavaDate` in `DateTimeUtils` was re-written by taking the implementation from Spark 2.4, and by rebasing the final results via `rebaseJulianToGregorianDays()`.

Also I updated `DateTimeBenchmark`, and added a benchmark for conversion from `java.sql.Date`.

### Why are the changes needed?
The PR fixes the regression of parallelizing a collection of `java.sql.Date` values, and improves performance of converting external values to Catalyst's `DATE` values:
- x4 on the master branch
- 30% against Spark 2.4.6-SNAPSHOT

Spark 2.4.6-SNAPSHOT:
```
To/from java.sql.Timestamp:               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Date                                  614            655          43          8.1         122.8       1.0X
```

Before the changes:
```
To/from java.sql.Timestamp:               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Date                                 1154           1206          46          4.3         230.9       1.0X
```

After:
```
To/from java.sql.Timestamp:               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Date                                  427            434           7         11.7          85.3       1.0X
```

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

### How was this patch tested?
- By existing tests suites, in particular, `DateTimeUtilsSuite`, `RebaseDateTimeSuite`, `DateFunctionsSuite`, `DateExpressionsSuite`.
- Re-run `DateTimeBenchmark` in the environment:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_242 and OpenJDK 64-Bit Server VM 11.0.6+10 |

Closes #28205 from MaxGekk/optimize-fromJavaDate.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-14 14:44:00 +00:00
yi.wu 5d4f5d36a2 [SPARK-30953][SQL] InsertAdaptiveSparkPlan should apply AQE on child plan of write commands
### What changes were proposed in this pull request?

This PR changes `InsertAdaptiveSparkPlan` to apply AQE on the child plan of V1/V2 write commands rather than the command itself.

### Why are the changes needed?

Apply AQE on write commands with child plan will expose `LogicalQueryStage` to `Analyzer` while it should hider under `AdaptiveSparkPlanExec` only to avoid unexpected broken.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #27701 from Ngone51/skip_v2_commands.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-14 05:18:58 +00:00
Max Gekk a0f8cc08a3 [SPARK-31426][SQL] Fix perf regressions of toJavaTimestamp/fromJavaTimestamp
### What changes were proposed in this pull request?
Reuse the `rebaseGregorianToJulianMicros()` and `rebaseJulianToGregorianMicros()` functions introduced by the PR #28119 in `DateTimeUtils`.`toJavaTimestamp()` and `fromJavaTimestamp()`. Actually, new implementation is derived from Spark 2.4 + rebasing via pre-calculated rebasing maps.

### Why are the changes needed?
The changes speed up conversions to/from java.sql.Timestamp, and as a consequence the PR improve performance of ORC datasource in loading/saving timestamps:
- Saving ~ **x2.8 faster** in master, and -11% against Spark 2.4.6
- Loading - **x3.2-4.5 faster** in master, -5% against Spark 2.4.6

Before:
```
Save timestamps to ORC:                   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
after 1582                                        59877          59877           0          1.7         598.8       0.0X
before 1582                                       61361          61361           0          1.6         613.6       0.0X

Load timestamps from ORC:                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
after 1582, vec off                               48197          48288         118          2.1         482.0       1.0X
after 1582, vec on                                38247          38351         128          2.6         382.5       1.3X
before 1582, vec off                              53179          53359         249          1.9         531.8       0.9X
before 1582, vec on                               44076          44268         269          2.3         440.8       1.1X
```

After:
```
Save timestamps to ORC:                   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
after 1582                                        21250          21250           0          4.7         212.5       0.1X
before 1582                                       22105          22105           0          4.5         221.0       0.1X

Load timestamps from ORC:                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
after 1582, vec off                               14903          14933          40          6.7         149.0       1.0X
after 1582, vec on                                 8342           8426          73         12.0          83.4       1.8X
before 1582, vec off                              15528          15575          76          6.4         155.3       1.0X
before 1582, vec on                                9025           9075          61         11.1          90.2       1.7X
```

Spark 2.4.6-SNAPSHOT:
```
Save timestamps to ORC:                   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
after 1582                                        18858          18858           0          5.3         188.6       1.0X
before 1582                                       18508          18508           0          5.4         185.1       1.0X

Load timestamps from ORC:                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
after 1582, vec off                               14063          14177         143          7.1         140.6       1.0X
after 1582, vec on                                 5955           6029         100         16.8          59.5       2.4X
before 1582, vec off                              14119          14126           7          7.1         141.2       1.0X
before 1582, vec on                                5991           6007          25         16.7          59.9       2.3X
```

### Does this PR introduce any user-facing change?
Yes, the `to_utc_timestamp` function returns the later local timestamp in the case of overlapping local timestamps at daylight saving time. it's changed back to the 2.4 behavior.

### How was this patch tested?
- By existing test suite `DateTimeUtilsSuite`, `RebaseDateTimeSuite`, `DateFunctionsSuite`, `DateExpressionsSuites`, `ParquetIOSuite`, `OrcHadoopFsRelationSuite`.
- Re-generating results of the benchmarks `DateTimeBenchmark` and `DateTimeRebaseBenchmark` in the environment:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_242 and OpenJDK 64-Bit Server VM 11.0.6+10 |

Closes #28189 from MaxGekk/optimize-to-from-java-timestamp.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-14 04:50:20 +00:00
Kent Yao 31b907748d [SPARK-31414][SQL][DOCS][FOLLOWUP] Update default datetime pattern for json/csv APIs documentations
### What changes were proposed in this pull request?

Update default datetime pattern from `yyyy-MM-dd'T'HH:mm:ss.SSSXXX ` to `yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX] ` for JSON/CSV APIs documentations

### Why are the changes needed?

doc fix

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

Yes, the documentation will change

### How was this patch tested?

Passing Jenkins

Closes #28204 from yaooqinn/SPARK-31414-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-14 10:25:37 +09:00
yi.wu bbb3cd9c5e [SPARK-31391][SQL][TEST] Add AdaptiveTestUtils to ease the test of AQE
### What changes were proposed in this pull request?

This PR adds `AdaptiveTestUtils` to make AQE test simpler, which includes:

`DisableAdaptiveExecution` - a test tag to skip a single test case if AQE is enabled.
`EnableAdaptiveExecutionSuite` - a helper trait to enable AQE for all tests except those tagged with `DisableAdaptiveExecution`.
`DisableAdaptiveExecutionSuite` - a helper trait to disable AQE for all tests.
`assertExceptionMessage` - a method to handle message of normal or AQE exception in a consistent way.
`assertExceptionCause` - a method to handle cause of normal or AQE exception in a consistent way.

### Why are the changes needed?

With this utils, we can:
- reduce much more duplicate codes;
- handle normal or AQE exception in a consistent way;
- improve the stability of AQE tests;

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

No.

### How was this patch tested?

Updated tests with the util.

Closes #28162 from Ngone51/add_aqe_test_utils.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-13 14:40:53 +00:00
yi.wu f6512903da [SPARK-31409][SQL][TEST] Fix failed tests due to result order changing when enable AQE
### What changes were proposed in this pull request?

This PR fix two tests by avoid result order changing when we enable AQE:

1. In `SQLQueryTestSuite`, disable BHJ optimization to avoid changing result order

2. In test `SQLQuerySuite#check outputs of expression examples`, disable  `spark.sql.adaptive.coalescePartitions.enabled`  to avoid changing result order

### Why are the changes needed?

query 147 in SQLQueryTestSuite#"udf/postgreSQL/udf-join.sql - Scala UDF" and test sql/SQLQuerySuite#"check outputs of expression examples" can fail when enable AQE due to result order changing. And this PR fix them.

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

No.

### How was this patch tested?

Tested manually with AQE enabled.

Closes #28178 from Ngone51/fix_order.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-13 14:36:25 +00:00
Max Gekk cac8d1b352 [SPARK-31398][SQL] Fix perf regression of loading dates before 1582 year by non-vectorized ORC reader
### What changes were proposed in this pull request?
In regular ORC reader when `spark.sql.orc.enableVectorizedReader` is set to `false`, I propose to use `DaysWritable` in reading DATE values from ORC files. Currently, days from ORC files are converted to java.sql.Date, and then to days in Proleptic Gregorian calendar. So, the conversion to Java type can be eliminated.

### Why are the changes needed?
- The PR fixes regressions in loading dates before the 1582 year from ORC files by when vectorised ORC reader is off.
- The changes improve performance of regular ORC reader for DATE columns.
  - x3.6 faster comparing to the current master
  - x1.9-x4.3 faster against Spark 2.4.6

Before (on JDK 8):
```
Load dates from ORC:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
after 1582, vec off                               39651          39686          31          2.5         396.5       1.0X
after 1582, vec on                                 3647           3660          13         27.4          36.5      10.9X
before 1582, vec off                              38155          38219          61          2.6         381.6       1.0X
before 1582, vec on                                4041           4046           6         24.7          40.4       9.8X
```

After (on JDK 8):
```
Load dates from ORC:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
after 1582, vec off                               10947          10971          28          9.1         109.5       1.0X
after 1582, vec on                                 3677           3702          36         27.2          36.8       3.0X
before 1582, vec off                              11456          11472          21          8.7         114.6       1.0X
before 1582, vec on                                4079           4103          21         24.5          40.8       2.7X
```

Spark 2.4.6:
```
Load dates from ORC:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
after 1582, vec off                               48169          48276          96          2.1         481.7       1.0X
after 1582, vec on                                 5375           5410          41         18.6          53.7       9.0X
before 1582, vec off                              22353          22482         198          4.5         223.5       2.2X
before 1582, vec on                                5474           5475           1         18.3          54.7       8.8X
```

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

### How was this patch tested?
- By existing tests suites like `DateTimeUtilsSuite`
- Checked for `hive-1.2` by:
```
./build/sbt -Phive-1.2 "test:testOnly *OrcHadoopFsRelationSuite"
```
- Re-run `DateTimeRebaseBenchmark` in the environment:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_242 and OpenJDK 64-Bit Server VM 11.0.6+10 |

Closes #28169 from MaxGekk/orc-optimize-dates.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-13 05:29:54 +00:00
Kent Yao d65f534c5a [SPARK-31414][SQL] Fix performance regression with new TimestampFormatter for json and csv time parsing
### What changes were proposed in this pull request?

With benchmark original, where the timestamp values are valid to the new parser

the result is
```scala
[info] Running benchmark: Read dates and timestamps
[info]   Running case: timestamp strings
[info]   Stopped after 3 iterations, 5781 ms
[info]   Running case: parse timestamps from Dataset[String]
[info]   Stopped after 3 iterations, 44764 ms
[info]   Running case: infer timestamps from Dataset[String]
[info]   Stopped after 3 iterations, 93764 ms
[info]   Running case: from_json(timestamp)
[info]   Stopped after 3 iterations, 59021 ms
```
When we modify the benchmark to

```scala
     def timestampStr: Dataset[String] = {
        spark.range(0, rowsNum, 1, 1).mapPartitions { iter =>
          iter.map(i => s"""{"timestamp":"1970-01-01T01:02:03.${i % 100}"}""")
        }.select($"value".as("timestamp")).as[String]
      }

      readBench.addCase("timestamp strings", numIters) { _ =>
        timestampStr.noop()
      }

      readBench.addCase("parse timestamps from Dataset[String]", numIters) { _ =>
        spark.read.schema(tsSchema).json(timestampStr).noop()
      }

      readBench.addCase("infer timestamps from Dataset[String]", numIters) { _ =>
        spark.read.json(timestampStr).noop()
      }
```
where the timestamp values are invalid for the new parser which causes a fallback to legacy parser(2.4).
the result is

```scala
[info] Running benchmark: Read dates and timestamps
[info]   Running case: timestamp strings
[info]   Stopped after 3 iterations, 5623 ms
[info]   Running case: parse timestamps from Dataset[String]
[info]   Stopped after 3 iterations, 506637 ms
[info]   Running case: infer timestamps from Dataset[String]
[info]   Stopped after 3 iterations, 509076 ms
```
About 10x perf-regression

BUT if we modify the timestamp pattern to `....HH:mm:ss[.SSS][XXX]` which make all timestamp values valid for the new parser to prohibit fallback, the result is

```scala
[info] Running benchmark: Read dates and timestamps
[info]   Running case: timestamp strings
[info]   Stopped after 3 iterations, 5623 ms
[info]   Running case: parse timestamps from Dataset[String]
[info]   Stopped after 3 iterations, 506637 ms
[info]   Running case: infer timestamps from Dataset[String]
[info]   Stopped after 3 iterations, 509076 ms
```

### Why are the changes needed?

 Fix performance regression.

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

NO
### How was this patch tested?

new tests added.

Closes #28181 from yaooqinn/SPARK-31414.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-13 03:11:28 +00:00
Kousuke Saruta 6cd0bef7fe
[SPARK-31416][SQL] Check more strictly that a field name can be used as a valid Java identifier for codegen
### What changes were proposed in this pull request?

Check more strictly that a field name can be used as a valid Java identifier in `ScalaReflection.serializerFor`
To check that, `SourceVersion` is used so that we need not add reserved keywords to be checked manually for the future Java versions (e.g, underscore, var, yield), .

### Why are the changes needed?

In the current implementation, `enum` is not checked even though it's a reserved keyword.
Also, there are lots of characters and sequences of character including numeric literals but they are not checked.
So we can't get better error message with following code.
```
case class  Data(`0`: Int)
Seq(Data(1)).toDF.show

20/04/11 03:24:24 ERROR CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 43, Column 1: Expression "value_0 = value_3" is not a type
org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 43, Column 1: Expression "value_0 = value_3" is not a type

...

```

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

Yes. With this change and the code example above, we can get following error message.
```
java.lang.UnsupportedOperationException: `0` is not a valid identifier of Java and cannot be used as field name
- root class: "Data"

...
```

### How was this patch tested?

Add another assertion to existing test case.

Closes #28184 from sarutak/improve-identifier-check.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-12 13:14:41 -07:00
gatorsmile ad79ae11ba
[SPARK-31424][SQL] Rename AdaptiveSparkPlanHelper.collectInPlanAndSubqueries to collectWithSubqueries
### What changes were proposed in this pull request?
Like https://github.com/apache/spark/pull/28092, this PR is to rename `QueryPlan.collectInPlanAndSubqueries` in AdaptiveSparkPlanHelper to `collectWithSubqueries`

### Why are the changes needed?
The old name is too verbose. `QueryPlan` is internal but it's the core of catalyst and we'd better make the API name clearer before we release it.

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

### How was this patch tested?
N/A

Closes #28193 from gatorsmile/spark-31322.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-12 13:10:57 -07:00
Dilip Biswal f0e2fc37d1 [SPARK-25154][SQL] Support NOT IN sub-queries inside nested OR conditions
### What changes were proposed in this pull request?

Currently NOT IN subqueries (predicated null aware subquery) are not allowed inside OR expressions. We currently catch this condition in checkAnalysis and throw an error.

This PR enhances the subquery rewrite to support this type of queries.

Query
```SQL
SELECT * FROM s1 WHERE a > 5 or b NOT IN (SELECT c FROM s2);
```
Optimized Plan
```SQL
== Optimized Logical Plan ==
Project [a#3, b#4]
+- Filter ((a#3 > 5) || NOT exists#7)
   +- Join ExistenceJoin(exists#7), ((b#4 = c#5) || isnull((b#4 = c#5)))
      :- HiveTableRelation `default`.`s1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#3, b#4]
      +- Project [c#5]
         +- HiveTableRelation `default`.`s2`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c#5, d#6]
```
This is rework from #22141.
The original author of this PR is dilipbiswal.

Closes #22141

### Why are the changes needed?

For better usability.

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

No.

### How was this patch tested?

Added new tests in SQLQueryTestSuite, RewriteSubquerySuite and SubquerySuite.
Output from DB2 as a reference:
[nested-not-db2.txt](https://github.com/apache/spark/files/2299945/nested-not-db2.txt)

Closes #28158 from maropu/pr22141.

Lead-authored-by: Dilip Biswal <dkbiswal@gmail.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-04-11 08:28:11 +09:00
yi.wu 6cddda7847 [SPARK-31384][SQL] Fix NPE in OptimizeSkewedJoin
### What changes were proposed in this pull request?

1. Fix NPE in `OptimizeSkewedJoin`

2. prevent other potential NPE errors in AQE.

### Why are the changes needed?

When there's a `inputRDD` of a plan has 0 partition, rule `OptimizeSkewedJoin` can hit NPE error because this kind of RDD means a null `MapOutputStatistics` due to:

d98df7626b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala (L68-L69)

Thus, we should take care of such NPE errors in other places too.

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

No

### How was this patch tested?

Added a test.

Closes #28153 from Ngone51/npe.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-10 08:16:48 +00:00
Kent Yao a454510917 [SPARK-31392][SQL] Support CalendarInterval to be reflect to CalendarntervalType
### What changes were proposed in this pull request?

Since 3.0.0, we make CalendarInterval public for input, it's better for it to be inferred to CalendarIntervalType.
In the PR, we add a rule for CalendarInterval to be mapped to CalendarIntervalType in ScalaRelection, then records(e.g case class, tuples ...) contains interval fields are able to convert to a Dataframe.

### Why are the changes needed?

CalendarInterval is public but can not be used as input for Datafame.

```scala
scala> import org.apache.spark.unsafe.types.CalendarInterval
import org.apache.spark.unsafe.types.CalendarInterval

scala> Seq((1, new CalendarInterval(1, 2, 3))).toDF("a", "b")
java.lang.UnsupportedOperationException: Schema for type org.apache.spark.unsafe.types.CalendarInterval is not supported
  at org.apache.spark.sql.catalyst.ScalaReflection$.$anonfun$schemaFor$1(ScalaReflection.scala:735)
```

this should be supported as well as
```scala
scala> sql("select interval 2 month 1 day a")
res2: org.apache.spark.sql.DataFrame = [a: interval]
```
### Does this PR introduce any user-facing change?

Yes, records(e.g case class, tuples ...) contains interval fields are able to convert to a Dataframe
### How was this patch tested?

add uts

Closes #28165 from yaooqinn/SPARK-31392.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-10 07:34:01 +00:00
Gabor Somogyi 1354d2d0de [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
### What changes were proposed in this pull request?
When loading DataFrames from JDBC datasource with Kerberos authentication, remote executors (yarn-client/cluster etc. modes) fail to establish a connection due to lack of Kerberos ticket or ability to generate it.

This is a real issue when trying to ingest data from kerberized data sources (SQL Server, Oracle) in enterprise environment where exposing simple authentication access is not an option due to IT policy issues.

In this PR I've added MariaDB support (other supported databases will come in later PRs).

What this PR contains:
* Introduced `SecureConnectionProvider` and added basic secure functionalities
* Added `MariaDBConnectionProvider`
* Added `MariaDBConnectionProviderSuite`
* Added `MariaDBKrbIntegrationSuite` docker integration test
* Added some missing code documentation

### Why are the changes needed?
Missing JDBC kerberos support.

### Does this PR introduce any user-facing change?
Yes, now user is able to connect to MariaDB using kerberos.

### How was this patch tested?
* Additional + existing unit tests
* Additional + existing integration tests
* Test on cluster manually

Closes #28019 from gaborgsomogyi/SPARK-31021.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@apache.org>
2020-04-09 09:20:02 -07:00
gengjiaan 014d33570b [SPARK-31291][SQL][TEST] SQLQueryTestSuite: Sharing test data and test tables among multiple test cases
### What changes were proposed in this pull request?
`SQLQueryTestSuite` spend 35 minutes time to test.
I've listed the 10 test cases that took the longest time in the `SQL` module below.

Class | Spend time  ↑ | Failure | Skip | Pass | Total test case
-- | -- | -- | -- | -- | --
SQLQueryTestSuite | 35 minutes | 0 | 1 | 230 | 231
TPCDSQuerySuite | 3 minutes 8 seconds | 0 | 0 | 156 | 156
SQLQuerySuite | 2 minutes 52 seconds | 0 | 0 | 185 | 185
DynamicPartitionPruningSuiteAEOff | 1 minutes 52 seconds | 0 | 0 | 22 | 22
DataFrameFunctionsSuite | 1 minutes 37 seconds | 0 | 0 | 102 | 102
DynamicPartitionPruningSuiteAEOn | 1 minutes 24 seconds | 0 | 0 | 22 | 22
DataFrameSuite | 1 minutes 14 seconds | 0 | 2 | 157 | 159
SubquerySuite | 1 minutes 12 seconds | 0 | 1 | 70 | 71
SingleLevelAggregateHashMapSuite | 1 minutes 1 seconds | 0 | 0 | 50 | 50
DataFrameAggregateSuite | 59 seconds | 0 | 0 | 50 | 50

I checked the code of `SQLQueryTestSuite` and found `SQLQueryTestSuite` load test data repeatedly.
This PR will improve the performance of `SQLQueryTestSuite`.

The total time run `SQLQueryTestSuite` before and after this PR show below.
Before
No | Time
-- | --
1 | 20 minutes, 22 seconds
2 | 23 minutes, 21 seconds
3 | 21 minutes, 19 seconds
4 | 22 minutes, 26 seconds
5 | 20 minutes, 8 seconds

After
No | Time
-- | --
1 | 20 minutes, 52 seconds
2 | 20 minutes, 47 seconds
3 | 20 minutes, 7 seconds
4 | 21 minutes, 10 seconds
5 | 20 minutes, 4 seconds

### Why are the changes needed?
Improve the performance of `SQLQueryTestSuite`.

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

### How was this patch tested?
Jenkins test

Closes #28060 from beliefer/avoid-load-test-data-repeatedly.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-09 12:16:43 +00:00
Max Gekk e2d9399602 [SPARK-31359][SQL] Speed up timestamps rebasing
### What changes were proposed in this pull request?
In the PR, I propose to optimise the `DateTimeUtils`.`rebaseJulianToGregorianMicros()` and `rebaseGregorianToJulianMicros()` functions, and make them faster by using pre-calculated rebasing tables. This approach allows to avoid expensive conversions via local timestamps. For example, the `America/Los_Angeles` time zone has just a few time points when difference between Proleptic Gregorian calendar and the hybrid calendar (Julian + Gregorian since 1582-10-15) is changed in the time interval 0001-01-01 .. 2100-01-01:

| i | local  timestamp | Proleptic Greg. seconds | Hybrid (Julian+Greg) seconds | difference in minutes|
| -- | ------- |----|----| ---- |
|0|0001-01-01 00:00|-62135568422|-62135740800|-2872|
|1|0100-03-01 00:00|-59006333222|-59006419200|-1432|
|...|...|...|...|...|
|13|1582-10-15 00:00|-12219264422|-12219264000|7|
|14|1883-11-18 12:00|-2717640000|-2717640000|0|

The difference in microseconds between Proleptic and hybrid calendars for any local timestamp in time intervals `[local timestamp(i), local timestamp(i+1))`, and for any microseconds in the time interval `[Gregorian micros(i), Gregorian micros(i+1))` is the same. In this way, we can rebase an input micros by following the steps:
1. Look at the table, and find the time interval where the micros falls to
2. Take the difference between 2 calendars for this time interval
3. Add the difference to the input micros. The result is rebased microseconds that has the same local timestamp representation.

Here are details of the implementation:
- Pre-calculated tables are stored to JSON files `gregorian-julian-rebase-micros.json` and `julian-gregorian-rebase-micros.json` in the resource folder of `sql/catalyst`. The diffs and switch time points are stored as seconds, for example:
```json
[
  {
    "tz" : "America/Los_Angeles",
    "switches" : [ -62135740800, -59006419200, ... , -2717640000 ],
    "diffs" : [ 172378, 85978, ..., 0 ]
  }
]
```
  The JSON files are generated by 2 tests in `RebaseDateTimeSuite` - `generate 'gregorian-julian-rebase-micros.json'` and `generate 'julian-gregorian-rebase-micros.json'`. Both tests are disabled by default.
  The `switches` time points are ordered from old to recent timestamps. This condition is checked by the test `validate rebase records in JSON files` in `RebaseDateTimeSuite`. Also sizes of the `switches` and `diffs` arrays are the same (this is checked by the same test).

- The **_Asia/Tehran, Iran, Africa/Casablanca and Africa/El_Aaiun_** time zones weren't added to the JSON files, see [SPARK-31385](https://issues.apache.org/jira/browse/SPARK-31385)
- The rebase info from the JSON files is placed to hash tables - `gregJulianRebaseMap` and `julianGregRebaseMap`. I use `AnyRefMap` because it is almost 2 times faster than Scala's immutable Map. Also I tried `java.util.HashMap` but it has worse lookup time than `AnyRefMap` in our case.
The hash maps store the switch time points and diffs in microseconds precision to avoid conversions from microseconds to seconds in the runtime.

- I moved the code related to days and microseconds rebasing to the separate object `RebaseDateTime` to do not pollute `DateTimeUtils`. Tests related to date-time rebasing are moved to `RebaseDateTimeSuite` for the same reason.

- I placed rebasing via local timestamp to separate methods that require zone id as the first parameter assuming that the caller has zone id already. This allows to void unnecessary retrieving the default time zone. The methods are marked as `private[sql]` because they are used in `RebaseDateTimeSuite` as reference implementation.

- Modified the `rebaseGregorianToJulianMicros()` and `rebaseJulianToGregorianMicros()` methods in `RebaseDateTime` to look up the rebase tables first of all. If hash maps don't contain rebasing info for the given time zone id, the methods falls back to the implementation via local timestamps. This allows to support time zones specified as zone offsets like '-08:00'.

### Why are the changes needed?
To make timestamps rebasing faster:
- Saving timestamps to parquet files is ~ **x3.8 faster**
- Loading timestamps from parquet files is ~**x2.8 faster**.
- Loading timestamps by Vectorized reader ~**x4.6 faster**.

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

### How was this patch tested?
- Added the test `validate rebase records in JSON files` to `RebaseDateTimeSuite`. The test validates 2 json files from the resource folder - `gregorian-julian-rebase-micros.json` and `julian-gregorian-rebase-micros.json`, and it checks per each time zone records that
  - the number of switch points is equal to the number of diffs between calendars. If the numbers are different, this will violate the assumption made in `RebaseDateTime.rebaseMicros`.
  - swith points are ordered from old to recent timestamps. This pre-condition is required for linear search in the `rebaseMicros` function.
- Added the test `optimization of micros rebasing - Gregorian to Julian` to `RebaseDateTimeSuite` which iterates over timestamps from 0001-01-01 to 2100-01-01 with the steps 1 ± 0.5 months, and checks that optimised function `RebaseDateTime`.`rebaseGregorianToJulianMicros()` returns the same result as non-optimised one. The check is performed for the UTC, PST, CET, Africa/Dakar, America/Los_Angeles, Antarctica/Vostok, Asia/Hong_Kong, Europe/Amsterdam time zones.
- Added the test `optimization of micros rebasing - Julian to Gregorian` to `RebaseDateTimeSuite` which does similar checks as the test above but for rebasing from the hybrid calendar (Julian + Gregorian) to Proleptic Gregorian calendar.
- The tests for days rebasing are moved from `DateTimeUtilsSuite` to `RebaseDateTimeSuite` because the rebasing related code is moved from `DateTimeUtils` to the separate object `RebaseDateTime`.
- Re-run `DateTimeRebaseBenchmark` at the America/Los_Angeles time zone (it is set explicitly in the PR #28127):

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_242 and OpenJDK 64-Bit Server VM 11.0.6+10 |

Closes #28119 from MaxGekk/optimize-rebase-micros.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-09 05:23:52 +00:00
Jungtaek Lim (HeartSaVioR) ca2ba4fe64 [SPARK-29314][SS] Don't overwrite the metric "updated" of state operator to 0 if empty batch is run
### What changes were proposed in this pull request?

This patch fixes the behavior of ProgressReporter which always overwrite the value of "updated" of state operator to 0 if there's no new data. The behavior is correct only when we copy the state progress from "previous" executed plan, meaning no batch has been run. (Nonzero value of "updated" would be odd if batch didn't run, so it was correct.)

It was safe to assume no data is no batch, but SPARK-24156 enables empty data can run the batch if Spark needs to deal with watermark. After the patch, it only overwrites the value if both two conditions are met: 1) no data 2) no batch.

### Why are the changes needed?

Currently Spark doesn't reflect correct metrics when empty batch is run and this patch fixes it.

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

No.

### How was this patch tested?

Modified UT. Note that FlatMapGroupsWithState increases the value of "updated" when state rows are removed.
Also manually tested via below query (not a simple query to test with spark-shell, as you'll meet closure issue in spark-shell while playing with state func):

> query

```
case class RunningCount(count: Long)

object TestFlatMapGroupsWithState {
  def main(args: Array[String]): Unit = {
    import org.apache.spark.sql.SparkSession

    val ss = SparkSession
      .builder()
      .appName("TestFlatMapGroupsWithState")
      .getOrCreate()

    ss.conf.set("spark.sql.shuffle.partitions", "5")

    import ss.implicits._

    val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => {
      if (state.hasTimedOut) {
        // End users are not restricted to remove the state here - they can update the
        // state as well. For example, event time session window would have list of
        // sessions here and it cannot remove entire state.
        state.update(RunningCount(-1))
        Iterator((key, "-1"))
      } else {
        val count = state.getOption.map(_.count).getOrElse(0L) + values.size
        state.update(RunningCount(count))
        state.setTimeoutDuration("1 seconds")
        Iterator((key, count.toString))
      }
    }

    implicit val sqlContext = ss.sqlContext
    val inputData = MemoryStream[String]

    val result = inputData
      .toDF()
      .as[String]
      .groupByKey { v => v }
      .flatMapGroupsWithState(OutputMode.Append(), GroupStateTimeout.ProcessingTimeTimeout())(stateFunc)

    val query = result
      .writeStream
      .format("memory")
      .option("queryName", "test")
      .outputMode("append")
      .trigger(Trigger.ProcessingTime("5 second"))
      .start()

    Thread.sleep(1000)

    var chIdx: Long = 0

    while (true) {
      (chIdx to chIdx + 4).map { idx => inputData.addData(idx.toString) }
      chIdx += 5
      // intentionally sleep much more than trigger to enable "empty" batch
      Thread.sleep(10 * 1000)
    }
  }
}
```

> before the patch (batch 3 which was an "empty" batch)

```
{
   "id":"de945a5c-882b-4dae-aa58-cb8261cbaf9e",
   "runId":"f1eb6d0d-3cd5-48b2-a03b-5e989b6c151b",
   "name":"test",
   "timestamp":"2019-11-18T07:00:25.005Z",
   "batchId":3,
   "numInputRows":0,
   "inputRowsPerSecond":0.0,
   "processedRowsPerSecond":0.0,
   "durationMs":{
      "addBatch":1664,
      "getBatch":0,
      "latestOffset":0,
      "queryPlanning":29,
      "triggerExecution":1789,
      "walCommit":51
   },
   "stateOperators":[
      {
         "numRowsTotal":10,
         "numRowsUpdated":0,
         "memoryUsedBytes":5130,
         "customMetrics":{
            "loadedMapCacheHitCount":15,
            "loadedMapCacheMissCount":0,
            "stateOnCurrentVersionSizeBytes":2722
         }
      }
   ],
   "sources":[
      {
         "description":"MemoryStream[value#1]",
         "startOffset":9,
         "endOffset":9,
         "numInputRows":0,
         "inputRowsPerSecond":0.0,
         "processedRowsPerSecond":0.0
      }
   ],
   "sink":{
      "description":"MemorySink",
      "numOutputRows":5
   }
}
```

> after the patch (batch 3 which was an "empty" batch)

```
{
   "id":"7cb41623-6b9a-408e-ae02-6796ec636fa0",
   "runId":"17847710-ddfe-45f5-a7ab-b160e149382f",
   "name":"test",
   "timestamp":"2019-11-18T07:02:25.005Z",
   "batchId":3,
   "numInputRows":0,
   "inputRowsPerSecond":0.0,
   "processedRowsPerSecond":0.0,
   "durationMs":{
      "addBatch":1196,
      "getBatch":0,
      "latestOffset":0,
      "queryPlanning":30,
      "triggerExecution":1333,
      "walCommit":46
   },
   "stateOperators":[
      {
         "numRowsTotal":10,
         "numRowsUpdated":5,
         "memoryUsedBytes":5130,
         "customMetrics":{
            "loadedMapCacheHitCount":15,
            "loadedMapCacheMissCount":0,
            "stateOnCurrentVersionSizeBytes":2722
         }
      }
   ],
   "sources":[
      {
         "description":"MemoryStream[value#1]",
         "startOffset":9,
         "endOffset":9,
         "numInputRows":0,
         "inputRowsPerSecond":0.0,
         "processedRowsPerSecond":0.0
      }
   ],
   "sink":{
      "description":"MemorySink",
      "numOutputRows":5
   }
}
```

"numRowsUpdated" is `0` in "stateOperators" before applying the patch which is "wrong", as we "update" the state when timeout occurs. After applying the patch, it correctly represents the "numRowsUpdated" as `5` in "stateOperators".

Closes #25987 from HeartSaVioR/SPARK-29314.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2020-04-08 16:59:39 -07:00
iRakson b56242332d
[SPARK-31009][SQL] Support json_object_keys function
### What changes were proposed in this pull request?
A new function `json_object_keys` is proposed in this PR. This function will return all the keys of the outmost json object. It takes Json Object as an argument.

- If invalid json expression is given, `NULL` will be returned.
- If an empty string or json array is given, `NULL` will be returned.
- If valid json object is given, all the keys of the outmost object will be returned as an array.
- For empty json object, empty array is returned.

We can also get JSON object keys using `map_keys+from_json`.  But `json_object_keys` is more efficient.
```
Performance result for json_object = {"a":[1,2,3,4,5], "b":[2,4,5,12333321]}

Intel(R) Core(TM) i7-9750H CPU  2.60GHz
JSON functions:                           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
json_object_keys                                  11666          12361         673          0.9        1166.6       1.0X
from_json+map_keys                                15309          15973         701          0.7        1530.9       0.8X

```

### Why are the changes needed?
This function will help naive users in directly extracting the keys from json string and its fairly intuitive as well. Also its extends the functionality of spark-sql for json strings.

Some of the most popular DBMSs supports this function.
- PostgreSQL
- MySQL
- MariaDB

### Does this PR introduce any user-facing change?
Yes. Now users can extract keys of json objects using this function.

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

Closes #27836 from iRakson/jsonKeys.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-08 13:04:59 -07:00
Burak Yavuz 8ab2a0c5f2 [SPARK-31278][SS] Fix StreamingQuery output rows metric
### What changes were proposed in this pull request?

In Structured Streaming, we provide progress updates every 10 seconds when a stream doesn't have any new data upstream. When providing this progress though, we zero out the input information but not the output information. This PR fixes that bug.

### Why are the changes needed?

Fixes a bug around incorrect metrics

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

Fixes a bug in the metrics

### How was this patch tested?

New regression test

Closes #28040 from brkyvz/sinkMetrics.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2020-04-07 17:17:47 -07:00
iRakson 71022d7130
[SPARK-31008][SQL] Support json_array_length function
### What changes were proposed in this pull request?
At the moment we do not have any function to compute length of JSON array directly.
I propose a  `json_array_length` function which will return the length of the outermost JSON array.

- This function will return length of the outermost JSON array, if JSON array is valid.
```

scala> spark.sql("select json_array_length('[1,2,3,[33,44],{\"key\":[2,3,4]}]')").show
+--------------------------------------------------+
|json_array_length([1,2,3,[33,44],{"key":[2,3,4]}])|
+--------------------------------------------------+
|                                                 5|
+--------------------------------------------------+

scala> spark.sql("select json_array_length('[[1],[2,3]]')").show
+------------------------------+
|json_array_length([[1],[2,3]])|
+------------------------------+
|                             2|
+------------------------------+

```
- In case of any other valid JSON string, invalid JSON string or null array or `NULL` input , `NULL` will be returned.
```
scala> spark.sql("select json_array_length('')").show
+-------------------+
|json_array_length()|
+-------------------+
|               null|
+-------------------+
```

### Why are the changes needed?

- As mentioned in JIRA, this function is supported by presto, postgreSQL, redshift, SQLite, MySQL, MariaDB, IBM DB2.

- for better user experience and ease of use.

```
Performance Result for Json array - [1, 2, 3, 4]

Intel(R) Core(TM) i7-9750H CPU  2.60GHz
JSON functions:                           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
json_array_length                                  7728           7762          53          1.3         772.8       1.0X
size+from_json                                    12739          12895         199          0.8        1273.9       0.6X

```

### Does this PR introduce any user-facing change?
Yes, now users can get length of a json array by using `json_array_length`.

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

Closes #27759 from iRakson/jsonArrayLength.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-07 15:34:33 -07:00
Eric Wu a28ed86a38
[SPARK-31113][SQL] Add SHOW VIEWS command
### What changes were proposed in this pull request?
Previously, user can issue `SHOW TABLES` to get info of both tables and views.
This PR (SPARK-31113) implements `SHOW VIEWS` SQL command similar to HIVE to get views only.(https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-ShowViews)

**Hive** -- Only show view names
```
hive> SHOW VIEWS;
OK
view_1
view_2
...
```

**Spark(Hive-Compatible)** -- Only show view names, used in tests and `SparkSQLDriver` for CLI applications
```
SHOW VIEWS IN showdb;
view_1
view_2
...
```

**Spark** -- Show more information database/viewName/isTemporary
```
spark-sql> SHOW VIEWS;
userdb	view_1	false
userdb	view_2	false
...
```

### Why are the changes needed?
`SHOW VIEWS` command provides better granularity to only get information of views.

### Does this PR introduce any user-facing change?
Add new `SHOW VIEWS` SQL command

### How was this patch tested?
Add new test `show-views.sql` and pass existing tests

Closes #27897 from Eric5553/ShowViews.

Authored-by: Eric Wu <492960551@qq.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-07 09:25:01 -07:00
beliefer 18923329f1 [SPARK-31316][SQL] SQLQueryTestSuite: Display the total generate time for generated java code
### What changes were proposed in this pull request?
After my investigation, SQLQueryTestSuite spent a lot of time to generate java code.
This PR is related to https://github.com/apache/spark/pull/28081.
https://github.com/apache/spark/pull/28081 used to display compile time, but this PR used to display generate time.
This PR will add the following to `SQLQueryTestSuite`'s output.
```
=== Metrics of Whole Codegen ===

Total generate time: 82.640913862 seconds

Total compile time: 98.649663572 seconds
```

### Why are the changes needed?
Display the total generate time for generated java code.

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

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

Closes #28105 from beliefer/output-codegen-generation-time.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-06 06:02:02 +00:00
Liang-Chi Hsieh d782a1c456 [SPARK-31224][SQL] Add view support to SHOW CREATE TABLE
### What changes were proposed in this pull request?

For now `SHOW CREATE TABLE` command doesn't support views, but `SHOW CREATE TABLE AS SERDE` supports it. Since the views syntax are the same between Hive DDL and Spark DDL, we should be able to support views in both two commands.

This is Hive syntax for creating views:

```
CREATE VIEW [IF NOT EXISTS] [db_name.]view_name [(column_name [COMMENT column_comment], ...) ]
  [COMMENT view_comment]
  [TBLPROPERTIES (property_name = property_value, ...)]
  AS SELECT ...;
```

https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-CreateView

This is Spark syntax for creating views:

```
CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [IF NOT EXISTS [db_name.]view_name
    create_view_clauses
    AS query;
```
https://spark.apache.org/docs/3.0.0-preview/sql-ref-syntax-ddl-create-view.html

Looks like it is the same. We could support views in both commands.

This patch proposes to add views support to `SHOW CREATE TABLE`.

### Why are the changes needed?

To extend the view support of `SHOW CREATE TABLE`, so users can use `SHOW CREATE TABLE` to show Spark DDL for views.

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

Yes. `SHOW CREATE TABLE` can be used to show Spark DDL for views.

### How was this patch tested?

Unit tests.

Closes #27984 from viirya/spark-view.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-06 05:34:59 +00:00
Max Gekk 35e6a9deee [SPARK-31353][SQL] Set a time zone in DateTimeBenchmark and DateTimeRebaseBenchmark
### What changes were proposed in this pull request?
In the PR, I propose to set the `America/Los_Angeles` time zone in the date-time benchmarks `DateTimeBenchmark` and `DateTimeRebaseBenchmark` via `withDefaultTimeZone(LA)` and `withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> LA.getId)`.

The results of affected benchmarks was given on an Amazon EC2 instance w/ the configuration:
| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK8/11 |

### Why are the changes needed?
Performance of date-time functions can depend on the system JVM time zone or SQL config `spark.sql.session.timeZone`. The changes allow to avoid any fluctuations of benchmarks results related to time zones, and set a reliable baseline for future optimization.

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

### How was this patch tested?
By regenerating results of DateTimeBenchmark and DateTimeRebaseBenchmark.

Closes #28127 from MaxGekk/set-timezone-in-benchmarks.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-06 05:21:04 +00:00
gatorsmile 39a6f518cb
[SPARK-29554][SQL][FOLLOWUP] Update Auto-generated Alias Name for Version
### What changes were proposed in this pull request?
The auto-generated alias name of built-in function `version()` is `sparkversion()`. After this PR, it is updated to `version()`.

### Why are the changes needed?
Based on our auto-generated alias name convention for the built-in functions, the alias names should be consistent with the function names.

This built-in function `version` is added in the upcoming Spark 3.0. Thus, we should fix it before the release.

### Does this PR introduce any user-facing change?
Yes. Update the column name in schema if users do not specify the alias.

### How was this patch tested?
Added a test case.

Closes #28131 from gatorsmile/spark-29554followup.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-05 16:37:03 -07:00
Wenchen Fan 6b1ca886c0 [SPARK-31327][SQL] Write Spark version into Avro file metadata
### What changes were proposed in this pull request?

Write Spark version into Avro file metadata

### Why are the changes needed?

The version info is very useful for backward compatibility. This is also done in parquet/orc.

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

no

### How was this patch tested?

new test

Closes #28102 from cloud-fan/avro.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-03 12:43:33 +00:00
Maxim Gekk 820bb9985a [SPARK-31328][SQL] Fix rebasing of overlapped local timestamps during daylight saving time
### What changes were proposed in this pull request?
1. Fix the `rebaseGregorianToJulianMicros()` function in `DateTimeUtils` by passing the daylight saving offset associated with the input `micros` to the constructed instance of `GregorianCalendar`. The problem is in `cal.getTimeInMillis` which returns earliest instant in the case of local date-time overlaps, see https://github.com/AdoptOpenJDK/openjdk-jdk8u/blob/master/jdk/src/share/classes/java/util/GregorianCalendar.java#L2783-L2786 . I fixed the issue by keeping the standard zone offset as is, and set the DST offset only. I don't set `ZONE_OFFSET` because time zone resolution works differently in Java 8 and Java 7 time APIs. So, if I would set the standard zone offsets too, this could change the behavior, and rebasing won't give the same result as Spark 2.4.
2. Fix `rebaseJulianToGregorianMicros()` by changing resulted zoned date-time if `DST_OFFSET` is zero which means the input date-time has passed an autumn daylight savings cutover. So, I take the latest local timestamp out of 2 overlapped timestamps. Otherwise I return a zoned date-time w/o any modification because it is equal to calling the `withEarlierOffsetAtOverlap()` method, so, we can optimize the case.

### Why are the changes needed?
This fixes the bug of loosing of DST offset info in rebasing timestamps via local date-time. For example, there are 2 different timestamps in the `America/Los_Angeles` time zone: `2019-11-03T01:00:00-07:00` and `2019-11-03T01:00:00-08:00`, though they are mapped to the same local date-time `2019-11-03T01:00`, see
<img width="456" alt="Screen Shot 2020-04-02 at 10 19 24" src="https://user-images.githubusercontent.com/1580697/78245697-95a7da00-74f0-11ea-9eba-c08138851cb3.png">
Currently, the UTC timestamp `2019-11-03T09:00:00Z` is converted to `2019-11-03T01:00:00-08:00`, and then to `2019-11-03T01:00:00` (in the original calendar, for instance Proleptic Gregorian calendar) and back to the UTC timestamp `2019-11-03T08:00:00Z` (in the hybrid calendar - Gregorian for the timestamp). That's wrong because the local timestamp must be converted to the original timestamp `2019-11-03T09:00:00Z`.

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

### How was this patch tested?
- Added a test to `DateTimeUtilsSuite` which checks that rebased micros are the same as the input during DST. The result must be the same if Java 8 and 7 time API functions return the same time zone offsets.
- Run the following code to check that there is no difference between rebased and original micros for modern timestamps:
```scala
    test("rebasing differences") {
      withDefaultTimeZone(getZoneId("America/Los_Angeles")) {
        val start = instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0)
          .atZone(getZoneId("America/Los_Angeles"))
          .toInstant)
        val end = instantToMicros(LocalDateTime.of(2030, 1, 1, 0, 0, 0)
          .atZone(getZoneId("America/Los_Angeles"))
          .toInstant)

        var micros = start
        var diff = Long.MaxValue
        var counter = 0
        while (micros < end) {
          val rebased = rebaseGregorianToJulianMicros(micros)
          val curDiff = rebased - micros
          if (curDiff != diff) {
            counter += 1
            diff = curDiff
            val ldt = microsToInstant(micros).atZone(getZoneId("America/Los_Angeles")).toLocalDateTime
            println(s"local date-time = $ldt diff = ${diff / MICROS_PER_MINUTE} minutes")
          }
          micros += 30 * MICROS_PER_MINUTE
        }
        println(s"counter = $counter")
      }
    }
```
```
local date-time = 0001-01-01T00:00 diff = -2872 minutes
local date-time = 0100-03-01T00:00 diff = -1432 minutes
local date-time = 0200-03-01T00:00 diff = 7 minutes
local date-time = 0300-03-01T00:00 diff = 1447 minutes
local date-time = 0500-03-01T00:00 diff = 2887 minutes
local date-time = 0600-03-01T00:00 diff = 4327 minutes
local date-time = 0700-03-01T00:00 diff = 5767 minutes
local date-time = 0900-03-01T00:00 diff = 7207 minutes
local date-time = 1000-03-01T00:00 diff = 8647 minutes
local date-time = 1100-03-01T00:00 diff = 10087 minutes
local date-time = 1300-03-01T00:00 diff = 11527 minutes
local date-time = 1400-03-01T00:00 diff = 12967 minutes
local date-time = 1500-03-01T00:00 diff = 14407 minutes
local date-time = 1582-10-15T00:00 diff = 7 minutes
local date-time = 1883-11-18T12:22:58 diff = 0 minutes
counter = 15
```
The code is not added to `DateTimeUtilsSuite` because it takes > 30 seconds.
- By running the updated benchmark `DateTimeRebaseBenchmark` via the command:
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.DateTimeRebaseBenchmark"
```
in the environment:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 1.8.0_242-8u242/11.0.6+10 |

Closes #28101 from MaxGekk/fix-local-date-overlap.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-03 04:35:31 +00:00
Takeshi Yamamuro d98df7626b [SPARK-31325][SQL][WEB UI] Control a plan explain mode in the events of SQL listeners via SQLConf
### What changes were proposed in this pull request?

This PR intends to add a new SQL config for controlling a plan explain mode in the events of (e.g., `SparkListenerSQLExecutionStart` and `SparkListenerSQLAdaptiveExecutionUpdate`) SQL listeners. In the current master, the output of `QueryExecution.toString` (this is equivalent to the "extended" explain mode) is stored in these events. I think it is useful to control the content via `SQLConf`. For example, the query "Details" content (TPCDS q66 query) of a SQL tab in a Spark web UI will be changed as follows;

Before this PR:
![q66-extended](https://user-images.githubusercontent.com/692303/78211668-950b4580-74e8-11ea-90c6-db52d437534b.png)

After this PR:
![q66-formatted](https://user-images.githubusercontent.com/692303/78211674-9ccaea00-74e8-11ea-9d1d-43c7e2b0f314.png)

### Why are the changes needed?

For better usability.

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

Yes; since Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`.

### How was this patch tested?

Added unit tests.

Closes #28097 from maropu/SPARK-31325.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-04-02 21:09:16 -07:00
Wenchen Fan 2c39502e84 [SPARK-31253][SQL][FOLLOWUP] Add metrics to AQE shuffle reader
<!--
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.
-->
This is a followup of https://github.com/apache/spark/pull/28022, to address three issues:
1. Add an assert in `CustomShuffleReaderExec` to make sure the partitions specs are all `PartialMapperPartitionSpec` or none.
2. Do not use `lazy val` for `partitionDataSizeMetrics` and `skewedPartitionMetrics`, as they will be merged into `metrics`, and `lazy val` will be serialized.
3. mark `metrics` as `transient`, as it's only used at driver-side
4. move `FileUtils.byteCountToDisplaySize` to `logDebug`, to save some calculation if log level is above debug.

### 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.
-->
followup improvement

### 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.
-->
existing tests

Closes #28103 from cloud-fan/ui.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2020-04-02 16:02:47 -07:00
beliefer a9260d0349 [SPARK-31315][SQL] SQLQueryTestSuite: Display the total compile time for generated java code
### What changes were proposed in this pull request?
After my investigation, `SQLQueryTestSuite` spent a lot of time compiling the generated java code.
Take `group-by.sql` as an example.
At first, I added some debug log into `SQLQueryTestSuite`.
Please reference 92b6af740c/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala (L402)
The execution command is as follows:
`build/sbt "~sql/test-only *SQLQueryTestSuite -- -z group-by.sql"`
The output show below:
```
00:56:06.192 WARN org.apache.spark.sql.SQLQueryTestSuite: group-by.sql using configs: spark.sql.codegen.wholeStage=true. run time: 20604
00:56:13.719 WARN org.apache.spark.sql.SQLQueryTestSuite: group-by.sql using configs: spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY. run time: 7526
00:56:18.786 WARN org.apache.spark.sql.SQLQueryTestSuite: group-by.sql using configs: spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN. run time: 5066
```
According to the log, we know.

Config | Run time(ms)
-- | --
spark.sql.codegen.wholeStage=true | 20604
spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY | 7526
spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN | 5066

We should display the total compile time for generated java code.

This PR will add the following to `SQLQueryTestSuite`'s output.
```
=== Metrics of Whole Codegen ===
Total compile time: 80.564516529 seconds
```

Note: At first, I wanted to use `CodegenMetrics.METRIC_COMPILATION_TIME` to do this. After many experiments, I found that `CodegenMetrics.METRIC_COMPILATION_TIME` is only effective for a single test case, and cannot play a role in the whole life cycle of `SQLQueryTestSuite`.
I checked the type of  ` CodegenMetrics.METRIC_COMPILATION_TIME` is `Histogram` and the latter preserves 1028 elements.` Histogram` is a metric which calculates the distribution of a value.

### Why are the changes needed?
Display the total compile time for generated java code.

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

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

Closes #28081 from beliefer/output-codegen-compile-time.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-02 09:13:22 +00:00
Kent Yao 1ce584f6b7 [SPARK-31321][SQL] Remove SaveMode check in v2 FileWriteBuilder
### What changes were proposed in this pull request?

The `SaveMode` is resolved before we create `FileWriteBuilder` to build `BatchWrite`.

In https://github.com/apache/spark/pull/25876, we removed save mode for DSV2 from DataFrameWriter. So that the `mode` method is never used which makes `validateInputs` fail determinately without `mode` set.

### Why are the changes needed?
rm dead code.

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

no

### How was this patch tested?

existing tests.

Closes #28090 from yaooqinn/SPARK-31321.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-02 08:34:36 +00:00
Mukul Murthy 34abbb677d [SPARK-31324][SS] Include stream ID in the termination timeout error message
### What changes were proposed in this pull request?

This PR (SPARK-31324) aims to include stream ID in the error thrown when a stream does not stop() in time.

### Why are the changes needed?

https://github.com/apache/spark/pull/26771/ added a conf to set a requested timeout for stopping a stream, after which the stop() method throws. From seeing this in a production use case with several streams running, it's helpful to include which stream failed to stop in the error message.

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

If a stream times out when terminating, the error message now includes the stream ID.

Before:
`Stream Execution thread failed to stop within 2000 milliseconds (specified by spark.sql.streaming.stopTimeout). See the cause on what was being executed in the streaming query thread.`

After:
`Stream Execution thread for stream [id = 8513769d-b9d2-4902-9b36-3668bd022245, runId = 21ed8c35-9bfe-423f-853d-c022d91818bc] failed to stop within 2000 milliseconds (specified by spark.sql.streaming.stopTimeout). See the cause on what was being executed in the streaming query thread.`

### How was this patch tested?

Updated existing unit test

Closes #28095 from mukulmurthy/31324-id.

Authored-by: Mukul Murthy <mukul.murthy@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-02 12:37:58 +09:00
Wenchen Fan 09f036a14c
[SPARK-31322][SQL] rename QueryPlan.collectInPlanAndSubqueries to collectWithSubqueries
### What changes were proposed in this pull request?

rename `QueryPlan.collectInPlanAndSubqueries` to `collectWithSubqueries`

### Why are the changes needed?

The old name is too verbose. `QueryPlan` is internal but it's the core of catalyst and we'd better make the API name clearer before we release it.

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

no

### How was this patch tested?

N/A

Closes #28092 from cloud-fan/rename.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-01 12:04:40 -07:00
Max Gekk 91af87d34e [SPARK-31311][SQL][TESTS] Benchmark date-time rebasing in ORC datasource
### What changes were proposed in this pull request?
In the PR, I propose to add new benchmarks to `DateTimeRebaseBenchmark` for saving and loading dates/timestamps to/from ORC files. I extracted common code from the benchmark for Parquet datasource and place it to the methods `caseName()` and `getPath()`. Added benchmarks for ORC save/load dates before and after 1582-10-15 because an implementation may have different performance for dates before the Julian calendar cutover day, see #28067 as an example.

### Why are the changes needed?
To have the base line for future optimizations of `fromJavaDate()`/`toJavaDate()` and `toJavaTimestamp()`/`fromJavaTimestamp()` in `DateTimeUtils`. The methods are used while saving/loading dates/timestamps by ORC datasource.

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

### How was this patch tested?
By running the updated benchmark `DateTimeRebaseBenchmark` via the command:
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.DateTimeRebaseBenchmark"
```
in the environment:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 1.8.0_242-8u242/11.0.6+10 |

Closes #28076 from MaxGekk/rebase-benchmark-orc.

Lead-authored-by: Max Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-01 07:02:26 +00:00
Maxim Gekk c5323d2e8d [SPARK-31318][SQL] Split Parquet/Avro configs for rebasing dates/timestamps in read and in write
### What changes were proposed in this pull request?
In the PR, I propose to replace the following SQL configs:
1.  `spark.sql.legacy.parquet.rebaseDateTime.enabled` by
    - `spark.sql.legacy.parquet.rebaseDateTimeInWrite.enabled` (`false` by default). The config enables rebasing dates/timestamps while saving to Parquet files. If it is set to `true`, dates/timestamps are converted to local date-time in Proleptic Gregorian calendar, date-time fields are extracted, and used in building new local date-time in the hybrid calendar (Julian + Gregorian). The resulted local date-time is converted to days or microseconds since the epoch.
    - `spark.sql.legacy.parquet.rebaseDateTimeInRead.enabled` (`false` by default). The config enables rebasing of dates/timestamps in reading from Parquet files.
2. `spark.sql.legacy.avro.rebaseDateTime.enabled` by
    - `spark.sql.legacy.avro.rebaseDateTimeInWrite.enabled` (`false` by default). It enables dates/timestamps rebasing from Proleptic Gregorian calendar to the hybrid calendar via local date/timestamps.
    - `spark.sql.legacy.avro.rebaseDateTimeInRead.enabled` (`false` by default).  It enables rebasing dates/timestamps from the hybrid calendar to Proleptic Gregorian calendar in read. The rebasing is performed by converting micros/millis/days to a local date/timestamp in the source calendar, interpreting the resulted date/timestamp in the target calendar, and getting the number of micros/millis/days since the epoch 1970-01-01 00:00:00Z.

### Why are the changes needed?
This allows to load dates/timestamps saved by Spark 2.4, and save to Parquet/Avro files without rebasing. And the reverse use case - load data saved by Spark 3.0, and save it in the form which is compatible with Spark 2.4.

### Does this PR introduce any user-facing change?
Yes, users have to use new SQL configs. Old SQL configs are removed by the PR.

### How was this patch tested?
By existing test suites `AvroV1Suite`, `AvroV2Suite` and `ParquetIOSuite`.

Closes #28082 from MaxGekk/split-rebase-configs.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-01 04:56:05 +00:00
Wenchen Fan 34c7ec8e0c [SPARK-31253][SQL] Add metrics to AQE shuffle reader
<!--
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.
-->
Add SQL metrics to the AQE shuffle reader (`CustomShuffleReaderExec`)

### 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.
-->
to be more UI friendly

### 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.
-->
new test

Closes #28022 from cloud-fan/metrics.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2020-03-31 13:03:52 -07:00
yi.wu 590b9a0132 [SPARK-31010][SQL][FOLLOW-UP] Add Java UDF suggestion in error message of untyped Scala UDF
### What changes were proposed in this pull request?

Added Java UDF suggestion in the in error message of untyped Scala UDF.

### Why are the changes needed?

To help user migrate their use case from deprecate untyped Scala UDF to other supported UDF.

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

No. It haven't been released.

### How was this patch tested?

Pass Jenkins.

Closes #28070 from Ngone51/spark_31010.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-31 17:35:26 +00:00
Wenchen Fan 8b01473e8b [SPARK-31230][SQL] Use statement plans in DataFrameWriter(V2)
### What changes were proposed in this pull request?

Create statement plans in `DataFrameWriter(V2)`, like the SQL API.

### Why are the changes needed?

It's better to leave all the resolution work to the analyzer.

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

no

### How was this patch tested?

existing tests

Closes #27992 from cloud-fan/statement.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-31 23:19:46 +08:00
Maxim Gekk bb0b416f0b [SPARK-31297][SQL] Speed up dates rebasing
### What changes were proposed in this pull request?
In the PR, I propose to replace current implementation of the `rebaseGregorianToJulianDays()` and `rebaseJulianToGregorianDays()` functions in `DateTimeUtils` by new one which is based on the fact that difference between Proleptic Gregorian and the hybrid (Julian+Gregorian) calendars was changed only 14 times for entire supported range of valid dates `[0001-01-01, 9999-12-31]`:

| date | Proleptic Greg. days | Hybrid (Julian+Greg) days | diff|
| ---- | ----|----|----|
|0001-01-01|-719162|-719164|-2|
|0100-03-01|-682944|-682945|-1|
|0200-03-01|-646420|-646420|0|
|0300-03-01|-609896|-609895|1|
|0500-03-01|-536847|-536845|2|
|0600-03-01|-500323|-500320|3|
|0700-03-01|-463799|-463795|4|
|0900-03-01|-390750|-390745|5|
|1000-03-01|-354226|-354220|6|
|1100-03-01|-317702|-317695|7|
|1300-03-01|-244653|-244645|8|
|1400-03-01|-208129|-208120|9|
|1500-03-01|-171605|-171595|10|
|1582-10-15|-141427|-141427|0|

For the given days since the epoch, the proposed implementation finds the range of days which the input days belongs to, and adds the diff in days between calendars to the input. The result is rebased days since the epoch in the target calendar.

For example, if need to rebase -650000 days from Proleptic Gregorian calendar to the hybrid calendar. In that case, the input falls to the bucket [-682944, -646420), the diff associated with the range is -1. To get the rebased days in Julian calendar, we should add -1 to -650000, and the result is -650001.

### Why are the changes needed?
To make dates rebasing faster.

### Does this PR introduce any user-facing change?
No, the results should be the same for valid range of the `DATE` type `[0001-01-01, 9999-12-31]`.

### How was this patch tested?
- Added 2 tests to `DateTimeUtilsSuite` for the `rebaseGregorianToJulianDays()` and `rebaseJulianToGregorianDays()` functions. The tests check that results of old and new implementation (optimized version) are the same for all supported dates.
- Re-run `DateTimeRebaseBenchmark` on:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK8/11 |

Closes #28067 from MaxGekk/optimize-rebasing.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-31 17:38:47 +08:00
Ben Ryves fa37856710 [SPARK-31306][DOCS] update rand() function documentation to indicate exclusive upper bound
### What changes were proposed in this pull request?
A small documentation change to clarify that the `rand()` function produces values in `[0.0, 1.0)`.

### Why are the changes needed?
`rand()` uses `Rand()` - which generates values in [0, 1) ([documented here](a1dbcd13a3/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala (L71))). The existing documentation suggests that 1.0 is a possible value returned by rand (i.e for a distribution written as `X ~ U(a, b)`, x can be a or b, so `U[0.0, 1.0]` suggests the value returned could include 1.0).

### Does this PR introduce any user-facing change?
Only documentation changes.

### How was this patch tested?
Documentation changes only.

Closes #28071 from Smeb/master.

Authored-by: Ben Ryves <benjamin.ryves@getyourguide.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-31 15:16:17 +09:00
Dongjoon Hyun cda2e30e77
Revert "[SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery"
This reverts commit f376d24ea1.
2020-03-30 19:14:14 -07:00
Maxim Gekk a1dbcd13a3 [SPARK-31296][SQL][TESTS] Benchmark date-time rebasing in Parquet datasource
### What changes were proposed in this pull request?
In the PR, I propose to add new benchmark `DateTimeRebaseBenchmark` which should measure the performance of rebasing of dates/timestamps from/to to the hybrid calendar (Julian+Gregorian) to/from Proleptic Gregorian calendar:
1. In write, it saves separately dates and timestamps before and after 1582 year w/ and w/o rebasing.
2. In read, it loads previously saved parquet files by vectorized reader and by regular reader.

Here is the summary of benchmarking:
- Saving timestamps is **~6 times slower**
- Loading timestamps w/ vectorized **off** is **~4 times slower**
- Loading timestamps w/ vectorized **on** is **~10 times slower**

### Why are the changes needed?
To know the impact of date-time rebasing introduced by #27915, #27953, #27807.

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

### How was this patch tested?
Run the `DateTimeRebaseBenchmark` benchmark using Amazon EC2:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK8/11 |

Closes #28057 from MaxGekk/rebase-bechmark.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-30 16:46:31 +08:00
Oleksii Kachaiev 22bb6b0fdd [SPARK-30532] DataFrameStatFunctions to work with TABLE.COLUMN syntax
### What changes were proposed in this pull request?
`DataFrameStatFunctions` now works correctly with fully qualified column name (Table.Column syntax) by properly resolving the name instead of relying on field names from schema, notably:
* `approxQuantile`
* `freqItems`
* `cov`
* `corr`

(other functions from `DataFrameStatFunctions` already work correctly).

See code examples below.

### Why are the changes needed?
With current implementation some stat functions are impossible to use when joining datasets with similar column names.

### Does this PR introduce any user-facing change?
Yes. Before the change, the following code would fail with `AnalysisException`.

```scala
scala> val df1 = sc.parallelize(0 to 10).toDF("num").as("table1")
df1: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [num: int]

scala> val df2 = sc.parallelize(0 to 10).toDF("num").as("table2")
df2: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [num: int]

scala> val dfx = df2.crossJoin(df1)
dfx: org.apache.spark.sql.DataFrame = [num: int, num: int]

scala> dfx.stat.approxQuantile("table1.num", Array(0.1), 0.0)
res0: Array[Double] = Array(1.0)

scala> dfx.stat.corr("table1.num", "table2.num")
res1: Double = 1.0

scala> dfx.stat.cov("table1.num", "table2.num")
res2: Double = 11.0

scala> dfx.stat.freqItems(Array("table1.num", "table2.num"))
res3: org.apache.spark.sql.DataFrame = [table1.num_freqItems: array<int>, table2.num_freqItems: array<int>]
```

### How was this patch tested?
Corresponding unit tests are added to `DataFrameStatSuite.scala` (marked as "SPARK-30532").

Closes #27916 from kachayev/fix-spark-30532.

Authored-by: Oleksii Kachaiev <kachayev@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-30 13:20:57 +08:00
Maxim Gekk d2ff5c5bfb [SPARK-31286][SQL][DOC] Specify formats of time zone ID for JSON/CSV option and from/to_utc_timestamp
### What changes were proposed in this pull request?
In the PR, I propose to update the doc for the `timeZone` option in JSON/CSV datasources and for the `tz` parameter of the `from_utc_timestamp()`/`to_utc_timestamp()` functions, and to restrict format of config's values to 2 forms:
1. Geographical regions, such as `America/Los_Angeles`.
2. Fixed offsets - a fully resolved offset from UTC. For example, `-08:00`.

### Why are the changes needed?
Other formats such as three-letter time zone IDs are ambitious, and depend on the locale. For example, `CST` could be U.S. `Central Standard Time` and `China Standard Time`. Such formats have been already deprecated in JDK, see [Three-letter time zone IDs](https://docs.oracle.com/javase/8/docs/api/java/util/TimeZone.html).

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

### How was this patch tested?
By running `./dev/scalastyle`, and manual testing.

Closes #28051 from MaxGekk/doc-time-zone-option.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-30 12:20:11 +08:00
Kent Yao f376d24ea1
[SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
### What changes were proposed in this pull request?
```sql
scala> spark.sql(" select * from values(1), (2) t(key) where key in (select 1 as key where 1=0)").queryExecution
res15: org.apache.spark.sql.execution.QueryExecution =
== Parsed Logical Plan ==
'Project [*]
+- 'Filter 'key IN (list#39 [])
   :  +- Project [1 AS key#38]
   :     +- Filter (1 = 0)
   :        +- OneRowRelation
   +- 'SubqueryAlias t
      +- 'UnresolvedInlineTable [key], [List(1), List(2)]

== Analyzed Logical Plan ==
key: int
Project [key#40]
+- Filter key#40 IN (list#39 [])
   :  +- Project [1 AS key#38]
   :     +- Filter (1 = 0)
   :        +- OneRowRelation
   +- SubqueryAlias t
      +- LocalRelation [key#40]

== Optimized Logical Plan ==
Join LeftSemi, (key#40 = key#38)
:- LocalRelation [key#40]
+- LocalRelation <empty>, [key#38]

== Physical Plan ==
*(1) BroadcastHashJoin [key#40], [key#38], LeftSemi, BuildRight
:- *(1) LocalTableScan [key#40]
+- Br...
```

`LocalRelation <empty> ` should be able to propagate after subqueries are lift up to joins

### Why are the changes needed?

optimize query

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

no
### How was this patch tested?

add new tests

Closes #28043 from yaooqinn/SPARK-31280.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-29 11:32:22 -07:00
gatorsmile 3884455780 [SPARK-31087] [SQL] Add Back Multiple Removed APIs
### What changes were proposed in this pull request?

Based on the discussion in the mailing list [[Proposal] Modification to Spark's Semantic Versioning Policy](http://apache-spark-developers-list.1001551.n3.nabble.com/Proposal-Modification-to-Spark-s-Semantic-Versioning-Policy-td28938.html) , this PR is to add back the following APIs whose maintenance cost are relatively small.

- functions.toDegrees/toRadians
- functions.approxCountDistinct
- functions.monotonicallyIncreasingId
- Column.!==
- Dataset.explode
- Dataset.registerTempTable
- SQLContext.getOrCreate, setActive, clearActive, constructors

Below is the other removed APIs in the original PR, but not added back in this PR [https://issues.apache.org/jira/browse/SPARK-25908]:

- Remove some AccumulableInfo .apply() methods
- Remove non-label-specific multiclass precision/recall/fScore in favor of accuracy
- Remove unused Python StorageLevel constants
- Remove unused multiclass option in libsvm parsing
- Remove references to deprecated spark configs like spark.yarn.am.port
- Remove TaskContext.isRunningLocally
- Remove ShuffleMetrics.shuffle* methods
- Remove BaseReadWrite.context in favor of session

### Why are the changes needed?
Avoid breaking the APIs that are commonly used.

### Does this PR introduce any user-facing change?
Adding back the APIs that were removed in 3.0 branch does not introduce the user-facing changes, because Spark 3.0 has not been released.

### How was this patch tested?
Added a new test suite for these APIs.

Author: gatorsmile <gatorsmile@gmail.com>
Author: yi.wu <yi.wu@databricks.com>

Closes #27821 from gatorsmile/addAPIBackV2.
2020-03-28 22:05:16 -07:00
Zhenhua Wang 791d2ba346 [SPARK-31261][SQL] Avoid npe when reading bad csv input with columnNameCorruptRecord specified
### What changes were proposed in this pull request?

SPARK-25387 avoids npe for bad csv input, but when reading bad csv input with `columnNameCorruptRecord` specified, `getCurrentInput` is called and it still throws npe.

### Why are the changes needed?

Bug fix.

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

No.

### How was this patch tested?

Add a test.

Closes #28029 from wzhfy/corrupt_column_npe.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-29 13:30:14 +09:00
Kengo Seki 0b237bd615 [SPARK-31292][CORE][SQL] Replace toSet.toSeq with distinct for readability
### What changes were proposed in this pull request?

This PR replaces the method calls of `toSet.toSeq` with `distinct`.

### Why are the changes needed?

`toSet.toSeq` is intended to make its elements unique but a bit verbose. Using `distinct` instead is easier to understand and improves readability.

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

No

### How was this patch tested?

Tested with the existing unit tests and found no problem.

Closes #28062 from sekikn/SPARK-31292.

Authored-by: Kengo Seki <sekikn@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-03-29 08:48:08 +09:00
Dongjoon Hyun d025ddbaa7
[SPARK-31238][SPARK-31284][TEST][FOLLOWUP] Fix readResourceOrcFile to create a local file from resource
### What changes were proposed in this pull request?

This PR aims to copy a test resource file to a local file in `OrcTest` suite before reading it.

### Why are the changes needed?

SPARK-31238 and SPARK-31284 added test cases to access the resouce file in `sql/core` module from `sql/hive` module. In **Maven** test environment, this causes a failure.
```
- SPARK-31238: compatibility with Spark 2.4 in reading dates *** FAILED ***
java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI:
jar:file:/home/jenkins/workspace/spark-master-test-maven-hadoop-3.2-hive-2.3-jdk-11/sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar!/test-data/before_1582_date_v2_4.snappy.orc
```

```
- SPARK-31284: compatibility with Spark 2.4 in reading timestamps *** FAILED ***
java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI:
jar:file:/home/jenkins/workspace/spark-master-test-maven-hadoop-3.2-hive-2.3/sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar!/test-data/before_1582_ts_v2_4.snappy.orc
```

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

No

### How was this patch tested?

Pass the Jenkins with Maven.

Closes #28059 from dongjoon-hyun/SPARK-31238.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-27 18:44:53 -07:00
Wenchen Fan c4e98c065c
[SPARK-31271][UI] fix web ui for driver side SQL metrics
### What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/23551, we changed the metrics type of driver-side SQL metrics to size/time etc. which comes with max/min/median info.

This doesn't make sense for driver side SQL metrics as they have only one value. It makes the web UI hard to read:
![image](https://user-images.githubusercontent.com/3182036/77653892-42db9900-6fab-11ea-8e7f-92f763fa32ff.png)

This PR updates the SQL metrics UI to only display max/min/median if there are more than one metrics values:
![image](https://user-images.githubusercontent.com/3182036/77653975-5f77d100-6fab-11ea-849e-64c935377c8e.png)

### Why are the changes needed?

Makes the UI easier to read

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

no

### How was this patch tested?
manual test

Closes #28037 from cloud-fan/ui.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-27 15:45:35 -07:00
Liang-Chi Hsieh aa8776bb59
[SPARK-29721][SQL] Prune unnecessary nested fields from Generate without Project
### What changes were proposed in this pull request?

This patch proposes to prune unnecessary nested fields from Generate which has no Project on top of it.

### Why are the changes needed?

In Optimizer, we can prune nested columns from Project(projectList, Generate). However, unnecessary columns could still possibly be read in Generate, if no Project on top of it. We should prune it too.

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

No

### How was this patch tested?

Unit test.

Closes #27517 from viirya/SPARK-29721-2.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-27 10:47:21 -07:00
Maxim Gekk fc2a974e03
[SPARK-31284][SQL][TESTS] Check rebasing of timestamps in ORC datasource
### What changes were proposed in this pull request?
In the PR, I propose 2 tests to check that rebasing of timestamps from/to the hybrid calendar (Julian + Gregorian) to/from Proleptic Gregorian calendar works correctly.
1. The test `compatibility with Spark 2.4 in reading timestamps` load ORC file saved by Spark 2.4.5 via:
```shell
$ export TZ="America/Los_Angeles"
```
```scala
scala> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")

scala> val df = Seq("1001-01-01 01:02:03.123456").toDF("tsS").select($"tsS".cast("timestamp").as("ts"))
df: org.apache.spark.sql.DataFrame = [ts: timestamp]

scala> df.write.orc("/Users/maxim/tmp/before_1582/2_4_5_ts_orc")

scala> spark.read.orc("/Users/maxim/tmp/before_1582/2_4_5_ts_orc").show(false)
+--------------------------+
|ts                        |
+--------------------------+
|1001-01-01 01:02:03.123456|
+--------------------------+
```
2. The test `rebasing timestamps in write` is round trip test. Since the previous test confirms correct rebasing of timestamps in read. This test should pass only if rebasing works correctly in write.

### Why are the changes needed?
To guarantee that rebasing works correctly for timestamps in ORC datasource.

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

### How was this patch tested?
By running `OrcSourceSuite` for Hive 1.2 and 2.3 via the commands:
```
$ build/sbt -Phive-2.3 "test:testOnly *OrcSourceSuite"
```
and
```
$ build/sbt -Phive-1.2 "test:testOnly *OrcSourceSuite"
```

Closes #28047 from MaxGekk/rebase-ts-orc-test.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-27 09:06:59 -07:00
Maxim Gekk 9f0c010a5c [SPARK-31277][SQL][TESTS] Migrate DateTimeTestUtils from TimeZone to ZoneId
### What changes were proposed in this pull request?
In the PR, I propose to change types of `DateTimeTestUtils` values and functions by replacing `java.util.TimeZone` to `java.time.ZoneId`. In particular:
1. Type of `ALL_TIMEZONES` is changed to `Seq[ZoneId]`.
2. Remove `val outstandingTimezones: Seq[TimeZone]`.
3. Change the type of the time zone parameter in `withDefaultTimeZone` to `ZoneId`.
4. Modify affected test suites.

### Why are the changes needed?
Currently, Spark SQL's date-time expressions and functions have been already ported on Java 8 time API but tests still use old time APIs. In particular, `DateTimeTestUtils` exposes functions that accept only TimeZone instances. This is inconvenient, and CPU consuming because need to convert TimeZone instances to ZoneId instances via strings (zone ids).

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

### How was this patch tested?
By affected test suites executed by jenkins builds.

Closes #28033 from MaxGekk/with-default-time-zone.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-27 21:14:25 +08:00
Kent Yao 5945d46c11 [SPARK-31225][SQL] Override sql method of OuterReference
### What changes were proposed in this pull request?

OuterReference is one LeafExpression, so it's children is Nil, which makes its SQL representation always be outer(). This makes our explain-command and error msg unclear when OuterReference exists.
e.g.

```scala
org.apache.spark.sql.AnalysisException:
Aggregate/Window/Generate expressions are not valid in where clause of the query.
Expression in where clause: [(in.`value` = max(outer()))]
Invalid expressions: [max(outer())];;
```
This PR override its `sql` method with its `prettyName` and single argment `e`'s `sql` methond

### Why are the changes needed?

improve err message

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

yes, the err msg caused by OuterReference has changed
### How was this patch tested?

modified ut results

Closes #27985 from yaooqinn/SPARK-31225.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-27 15:21:19 +08:00
gatorsmile b9eafcb526 [SPARK-31088][SQL] Add back HiveContext and createExternalTable
### What changes were proposed in this pull request?
Based on the discussion in the mailing list [[Proposal] Modification to Spark's Semantic Versioning Policy](http://apache-spark-developers-list.1001551.n3.nabble.com/Proposal-Modification-to-Spark-s-Semantic-Versioning-Policy-td28938.html) , this PR is to add back the following APIs whose maintenance cost are relatively small.

- HiveContext
- createExternalTable APIs

### Why are the changes needed?

Avoid breaking the APIs that are commonly used.

### Does this PR introduce any user-facing change?
Adding back the APIs that were removed in 3.0 branch does not introduce the user-facing changes, because Spark 3.0 has not been released.

### How was this patch tested?

add a new test suite for createExternalTable APIs.

Closes #27815 from gatorsmile/addAPIsBack.

Lead-authored-by: gatorsmile <gatorsmile@gmail.com>
Co-authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2020-03-26 23:51:15 -07:00
gatorsmile b7e4cc775b [SPARK-31086][SQL] Add Back the Deprecated SQLContext methods
### What changes were proposed in this pull request?

Based on the discussion in the mailing list [[Proposal] Modification to Spark's Semantic Versioning Policy](http://apache-spark-developers-list.1001551.n3.nabble.com/Proposal-Modification-to-Spark-s-Semantic-Versioning-Policy-td28938.html) , this PR is to add back the following APIs whose maintenance cost are relatively small.

- SQLContext.applySchema
- SQLContext.parquetFile
- SQLContext.jsonFile
- SQLContext.jsonRDD
- SQLContext.load
- SQLContext.jdbc

### Why are the changes needed?
Avoid breaking the APIs that are commonly used.

### Does this PR introduce any user-facing change?
Adding back the APIs that were removed in 3.0 branch does not introduce the user-facing changes, because Spark 3.0 has not been released.

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

Closes #27839 from gatorsmile/addAPIBackV3.

Lead-authored-by: gatorsmile <gatorsmile@gmail.com>
Co-authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2020-03-26 23:49:24 -07:00
DB Tsai cb0db21373 [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][TEST-HIVE1.2] Nested Column Predicate Pushdown for Parquet
### What changes were proposed in this pull request?
1. `DataSourceStrategy.scala` is extended to create `org.apache.spark.sql.sources.Filter` from nested expressions.
2. Translation from nested `org.apache.spark.sql.sources.Filter` to `org.apache.parquet.filter2.predicate.FilterPredicate` is implemented to support nested predicate pushdown for Parquet.

### Why are the changes needed?
Better performance for handling nested predicate pushdown.

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

### How was this patch tested?
New tests are added.

Closes #27728 from dbtsai/SPARK-17636.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-27 14:28:57 +08:00
Kousuke Saruta bc37fdc771 [SPARK-31275][WEBUI] Improve the metrics format in ExecutionPage for StageId
### What changes were proposed in this pull request?

In ExecutionPage, metrics format for stageId, attemptId and taskId are displayed like `(stageId (attemptId): taskId)` for now.
I changed this format like `(stageId.attemptId taskId)`.

### Why are the changes needed?

As cloud-fan suggested  [here](https://github.com/apache/spark/pull/27927#discussion_r398591519), `stageId.attemptId` is more standard in Spark.

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

Yes. Before applying this change, we can see the UI like as follows.
![with-checked](https://user-images.githubusercontent.com/4736016/77682421-42a6c200-6fda-11ea-92e4-e9f4554adb71.png)

And after this change applied, we can like as follows.
![fix-merics-format-with-checked](https://user-images.githubusercontent.com/4736016/77682493-61a55400-6fda-11ea-801f-91a67da698fd.png)

### How was this patch tested?

Modified `SQLMetricsSuite` and manual test.

Closes #28039 from sarutak/improve-metrics-format.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-27 13:35:28 +08:00
Terry Kim a97d3b9f4f [SPARK-31204][SQL] HiveResult compatibility for DatasourceV2 command
### What changes were proposed in this pull request?

`HiveResult` performs some conversions for commands to be compatible with Hive output, e.g.:
```
// If it is a describe command for a Hive table, we want to have the output format be similar with Hive.
case ExecutedCommandExec(_: DescribeCommandBase) =>
...
// SHOW TABLES in Hive only output table names, while ours output database, table name, isTemp.
case command  ExecutedCommandExec(s: ShowTablesCommand) if !s.isExtended =>
```
This conversion is needed for DatasourceV2 commands as well and this PR proposes to add the conversion for v2 commands `SHOW TABLES` and `DESCRIBE TABLE`.

### Why are the changes needed?

This is a bug where conversion is not applied to v2 commands.

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

Yes, now the outputs for v2 commands `SHOW TABLES` and `DESCRIBE TABLE` are compatible with HIVE output.

For example, with a table created as:
```
CREATE TABLE testcat.ns.tbl (id bigint COMMENT 'col1') USING foo
```

The output of `SHOW TABLES` has changed from
```
ns    table
```
to
```
table
```

And the output of `DESCRIBE TABLE` has changed from
```
id    bigint    col1

# Partitioning
Not partitioned
```
to
```
id                      bigint                  col1

# Partitioning
Not partitioned
```

### How was this patch tested?

Added unit tests.

Closes #28004 from imback82/hive_result.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-27 12:48:14 +08:00
Kent Yao 8be16907c2 [SPARK-31170][SQL] Spark SQL Cli should respect hive-site.xml and spark.sql.warehouse.dir
### What changes were proposed in this pull request?

In Spark CLI, we create a hive `CliSessionState` and it does not load the `hive-site.xml`. So the configurations in `hive-site.xml` will not take effects like other spark-hive integration apps.

Also, the warehouse directory is not correctly picked. If the `default` database does not exist, the `CliSessionState` will create one during the first time it talks to the metastore. The `Location` of the default DB will be neither the value of `spark.sql.warehousr.dir` nor the user-specified value of `hive.metastore.warehourse.dir`, but the default value of `hive.metastore.warehourse.dir `which will always be `/user/hive/warehouse`.

This PR fixes CLiSuite failure with the hive-1.2 profile in https://github.com/apache/spark/pull/27933.

In https://github.com/apache/spark/pull/27933, we fix the issue in JIRA by deciding the warehouse dir using all properties from spark conf and Hadoop conf, but properties from `--hiveconf` is not included,  they will be applied to the `CliSessionState` instance after it initialized. When this command-line option key is `hive.metastore.warehouse.dir`, the actual warehouse dir is overridden. Because of the logic in Hive for creating the non-existing default database changed, that test passed with `Hive 2.3.6` but failed with `1.2`. So in this PR, Hadoop/Hive configurations are ordered by:
` spark.hive.xxx > spark.hadoop.xxx > --hiveconf xxx > hive-site.xml` througth `ShareState.loadHiveConfFile` before sessionState start

### Why are the changes needed?

Bugfix for Spark SQL CLI to pick right confs

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

yes,
1. the non-exists default database will be created in the location specified by the users via `spark.sql.warehouse.dir` or `hive.metastore.warehouse.dir`, or the default value of `spark.sql.warehouse.dir` if none of them specified.

2. configurations from `hive-site.xml` will not override command-line options or the properties defined with `spark.hadoo(hive).` prefix in spark conf.

### How was this patch tested?

add cli ut

Closes #27969 from yaooqinn/SPARK-31170-2.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-27 12:05:45 +08:00
beliefer 9e0fee933e [SPARK-31262][SQL][TESTS] Fix bug tests imported bracketed comments
### What changes were proposed in this pull request?
This PR related to https://github.com/apache/spark/pull/27481.
If test case A uses `--IMPORT` to import test case B contains bracketed comments, the output can't display bracketed comments in golden files well.
The content of `nested-comments.sql` show below:
```
-- This test case just used to test imported bracketed comments.

-- the first case of bracketed comment
--QUERY-DELIMITER-START
/* This is the first example of bracketed comment.
SELECT 'ommented out content' AS first;
*/
SELECT 'selected content' AS first;
--QUERY-DELIMITER-END
```
The test case `comments.sql` imports `nested-comments.sql` below:
`--IMPORT nested-comments.sql`
Before this PR, the output will be:
```
-- !query
/* This is the first example of bracketed comment.
SELECT 'ommented out content' AS first
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException

mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP',
'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', '
ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0)

== SQL ==
/* This is the first example of bracketed comment.
^^^
SELECT 'ommented out content' AS first

-- !query
*/
SELECT 'selected content' AS first
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException

extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0)

== SQL ==
*/
^^^
SELECT 'selected content' AS first
```
After this PR, the output will be:
```
-- !query
/* This is the first example of bracketed comment.
SELECT 'ommented out content' AS first;
*/
SELECT 'selected content' AS first
-- !query schema
struct<first:string>
-- !query output
selected content
```

### Why are the changes needed?
Golden files can't display the bracketed comments in imported test cases.

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

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

Closes #28018 from beliefer/fix-bug-tests-imported-bracketed-comments.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-03-27 08:09:17 +09:00
Maxim Gekk d72ec85741
[SPARK-31238][SQL] Rebase dates to/from Julian calendar in write/read for ORC datasource
### What changes were proposed in this pull request?

This PR (SPARK-31238) aims the followings.
1. Modified ORC Vectorized Reader, in particular, OrcColumnVector v1.2 and v2.3. After the changes, it uses `DateTimeUtils. rebaseJulianToGregorianDays()` added by https://github.com/apache/spark/pull/27915 . The method performs rebasing days from the hybrid calendar (Julian + Gregorian) to Proleptic Gregorian calendar. It builds a local date in the original calendar, extracts date fields `year`, `month` and `day` from the local date, and builds another local date in the target calendar. After that, it calculates days from the epoch `1970-01-01` for the resulted local date.
2. Introduced rebasing dates while saving ORC files, in particular, I modified `OrcShimUtils. getDateWritable` v1.2 and v2.3, and returned `DaysWritable` instead of Hive's `DateWritable`. The `DaysWritable` class was added by the PR https://github.com/apache/spark/pull/27890 (and fixed by https://github.com/apache/spark/pull/27962). I moved `DaysWritable` from `sql/hive` to `sql/core` to re-use it in ORC datasource.

### Why are the changes needed?
For the backward compatibility with Spark 2.4 and earlier versions. The changes allow users to read dates/timestamps saved by previous version, and get the same result.

### Does this PR introduce any user-facing change?
Yes. Before the changes, loading the date `1200-01-01` saved by Spark 2.4.5 returns the following:
```scala
scala> spark.read.orc("/Users/maxim/tmp/before_1582/2_4_5_date_orc").show(false)
+----------+
|dt        |
+----------+
|1200-01-08|
+----------+
```
After the changes
```scala
scala> spark.read.orc("/Users/maxim/tmp/before_1582/2_4_5_date_orc").show(false)
+----------+
|dt        |
+----------+
|1200-01-01|
+----------+
```

### How was this patch tested?
- By running `OrcSourceSuite` and `HiveOrcSourceSuite`.
- Add new test `SPARK-31238: compatibility with Spark 2.4 in reading dates` to `OrcSuite` which reads an ORC file saved by Spark 2.4.5 via the commands:
```shell
$ export TZ="America/Los_Angeles"
```
```scala
scala> sql("select cast('1200-01-01' as date) dt").write.mode("overwrite").orc("/Users/maxim/tmp/before_1582/2_4_5_date_orc")
scala> spark.read.orc("/Users/maxim/tmp/before_1582/2_4_5_date_orc").show(false)
+----------+
|dt        |
+----------+
|1200-01-01|
+----------+
```
- Add round trip test `SPARK-31238: rebasing dates in write`. The test `SPARK-31238: compatibility with Spark 2.4 in reading dates` confirms rebasing in read. So, we can check rebasing in write.

Closes #28016 from MaxGekk/rebase-date-orc.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-26 13:14:28 -07:00
Wenchen Fan 05498af72e [SPARK-31201][SQL] Add an individual config for skewed partition threshold
### What changes were proposed in this pull request?

Skew join handling comes with an overhead: we need to read some data repeatedly. We should treat a partition as skewed if it's large enough so that it's beneficial to do so.

Currently the size threshold is the advisory partition size, which is 64 MB by default. This is not large enough for the skewed partition size threshold.

This PR adds a new config for the threshold and set default value as 256 MB.

### Why are the changes needed?

Avoid skew join handling that may introduce a  perf regression.

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

no

### How was this patch tested?

existing tests

Closes #27967 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 22:57:01 +09:00
yi.wu 8b798c1bc5 [SPARK-31242][SQL][TEST] mergeSparkConf in WithTestConf should also respect spark.sql.legacy.sessionInitWithConfigDefaults
### What changes were proposed in this pull request?

Make `mergeSparkConf` in `WithTestConf` respects `spark.sql.legacy.sessionInitWithConfigDefaults`.

### Why are the changes needed?

Without the fix, conf specified by `withSQLConf` can be reverted to original value in a cloned SparkSession.  For example, you will fail test below without the fix:

```
withSQLConf(SQLConf.CODEGEN_FALLBACK.key -> "true") {
  val cloned = spark.cloneSession()
  SparkSession.setActiveSession(cloned)
  assert(SQLConf.get.getConf(SQLConf.CODEGEN_FALLBACK) === true)
}
```

So we should fix it just as  #24540 did before.

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

No.

### How was this patch tested?

Added tests.

Closes #28014 from Ngone51/sparksession_clone.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-26 18:52:56 +08:00
Maxim Gekk 600319dcb9 [SPARK-31254][SQL] Use the current session time zone in HiveResult.toHiveString
### What changes were proposed in this pull request?
In the PR, I propose to define `timestampFormatter`, `dateFormatter` and `zoneId` as methods of the `HiveResult` object. This should guarantee that the formatters pick the current session time zone in `toHiveString()`

### Why are the changes needed?
Currently, date/timestamp formatters in `HiveResult.toHiveString` are initialized once on instantiation of the `HiveResult` object, and pick up the session time zone. If the sessions time zone is changed, the formatters still use the previous one.

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

### How was this patch tested?
By existing test suites, in particular, by `HiveResultSuite`

Closes #28024 from MaxGekk/hive-result-datetime-formatters.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-26 17:45:29 +08:00
HyukjinKwon 3bd10ce007 [SPARK-31227][SQL] Non-nullable null type in complex types should not coerce to nullable type
### What changes were proposed in this pull request?

This PR targets for non-nullable null type not to coerce to nullable type in complex types.

Non-nullable fields in struct, elements in an array and entries in map can mean empty array, struct and map. They are empty so it does not need to force the nullability when we find common types.

This PR also reverts and supersedes d7b97a1d0d

### Why are the changes needed?

To make type coercion coherent and consistent. Currently, we correctly keep the nullability even between non-nullable fields:

```scala
import org.apache.spark.sql.types._
import org.apache.spark.sql.functions._
spark.range(1).select(array(lit(1)).cast(ArrayType(IntegerType, false))).printSchema()
spark.range(1).select(array(lit(1)).cast(ArrayType(DoubleType, false))).printSchema()
```
```scala
spark.range(1).selectExpr("concat(array(1), array(1)) as arr").printSchema()
```

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

Yes.

```scala
import org.apache.spark.sql.types._
import org.apache.spark.sql.functions._
spark.range(1).select(array().cast(ArrayType(IntegerType, false))).printSchema()
```
```scala
spark.range(1).selectExpr("concat(array(), array(1)) as arr").printSchema()
```

**Before:**

```
org.apache.spark.sql.AnalysisException: cannot resolve 'array()' due to data type mismatch: cannot cast array<null> to array<int>;;
'Project [cast(array() as array<int>) AS array()#68]
+- Range (0, 1, step=1, splits=Some(12))

  at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$$nestedInanonfun$checkAnalysis$1$2.applyOrElse(CheckAnalysis.scala:149)
  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$$nestedInanonfun$checkAnalysis$1$2.applyOrElse(CheckAnalysis.scala:140)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUp$2(TreeNode.scala:333)
  at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:333)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUp$1(TreeNode.scala:330)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:399)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:237)
```

```
root
 |-- arr: array (nullable = false)
 |    |-- element: integer (containsNull = true)
```

**After:**

```
root
 |-- array(): array (nullable = false)
 |    |-- element: integer (containsNull = false)
```

```
root
 |-- arr: array (nullable = false)
 |    |-- element: integer (containsNull = false)
```

### How was this patch tested?

Unittests were added and manually tested.

Closes #27991 from HyukjinKwon/SPARK-31227.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-26 15:42:54 +08:00
Kent Yao 44bd36ad7b [SPARK-31234][SQL] ResetCommand should reset config to sc.conf only
### What changes were proposed in this pull request?
Currently, ResetCommand clear all configurations, including sql configs, static sql configs and spark context level configs.
for example:
```sql
spark-sql> set xyz=abc;
xyz abc
spark-sql> set;
spark.app.id local-1585055396930
spark.app.name SparkSQL::10.242.189.214
spark.driver.host 10.242.189.214
spark.driver.port 65094
spark.executor.id driver
spark.jars
spark.master local[*]
spark.sql.catalogImplementation hive
spark.sql.hive.version 1.2.1
spark.submit.deployMode client
xyz abc
spark-sql> reset;
spark-sql> set;
spark-sql> set spark.sql.hive.version;
spark.sql.hive.version 1.2.1
spark-sql> set spark.app.id;
spark.app.id <undefined>
```
In this PR, we restore spark confs to  RuntimeConfig after it is cleared

### Why are the changes needed?
reset command overkills configs which are static.
### Does this PR introduce any user-facing change?

yes, the ResetCommand do not change static configs now

### How was this patch tested?

add ut

Closes #28003 from yaooqinn/SPARK-31234.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-26 15:03:16 +08:00
Maxim Gekk cec9604eae [SPARK-31237][SQL][TESTS] Replace 3-letter time zones by zone offsets
### What changes were proposed in this pull request?
In the PR, I propose to add a few `ZoneId` constant values to the `DateTimeTestUtils` object, and reuse the constants in tests. Proposed the following constants:
- PST = -08:00
- UTC = +00:00
- CEST = +02:00
- CET = +01:00
- JST = +09:00
- MIT = -09:30
- LA = America/Los_Angeles

### Why are the changes needed?
All proposed constant values (except `LA`) are initialized by zone offsets according to their definitions. This will allow to avoid:
- Using of 3-letter time zones that have been already deprecated in JDK, see _Three-letter time zone IDs_ in https://docs.oracle.com/javase/8/docs/api/java/util/TimeZone.html
- Incorrect mapping of 3-letter time zones to zone offsets, see SPARK-31237. For example, `PST` is mapped to `America/Los_Angeles` instead of the `-08:00` zone offset.

Also this should improve stability and maintainability of test suites.

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

### How was this patch tested?
By running affected test suites.

Closes #28001 from MaxGekk/replace-pst.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-26 13:36:00 +08:00
Wenchen Fan 4f274a4de9
[SPARK-31147][SQL] Forbid CHAR type in non-Hive-Serde tables
### What changes were proposed in this pull request?

Spark introduced CHAR type for hive compatibility but it only works for hive tables. CHAR type is never documented and is treated as STRING type for non-Hive tables.

However, this leads to confusing behaviors

**Apache Spark 3.0.0-preview2**
```
spark-sql> CREATE TABLE t(a CHAR(3));

spark-sql> INSERT INTO TABLE t SELECT 'a ';

spark-sql> SELECT a, length(a) FROM t;
a 	2
```

**Apache Spark 2.4.5**
```
spark-sql> CREATE TABLE t(a CHAR(3));

spark-sql> INSERT INTO TABLE t SELECT 'a ';

spark-sql> SELECT a, length(a) FROM t;
a  	3
```

According to the SQL standard, `CHAR(3)` should guarantee all the values are of length 3. Since `CHAR(3)` is treated as STRING so Spark doesn't guarantee it.

This PR forbids CHAR type in non-Hive tables as it's not supported correctly.

### Why are the changes needed?

avoid confusing/wrong behavior

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

yes, now users can't create/alter non-Hive tables with CHAR type.

### How was this patch tested?

new tests

Closes #27902 from cloud-fan/char.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-25 09:25:55 -07:00
samsetegne 44431d4b1a [SPARK-30822][SQL] Remove semicolon at the end of a sql query
# What changes were proposed in this pull request?
This change proposes ignoring a terminating semicolon from queries submitted by the user (if included) instead of raising a parse exception.

# Why are the changes needed?
When a user submits a directly executable SQL statement terminated with a semicolon, they receive an `org.apache.spark.sql.catalyst.parser.ParseException` of `extraneous input ';' expecting <EOF>`. SQL-92 describes a direct SQL statement as having the format of `<directly executable statement> <semicolon>` and the majority of SQL implementations either require the semicolon as a statement terminator, or make it optional (meaning not raising an exception when it's included, seemingly in recognition that it's a common behavior).

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

# How was this patch tested?
Unit test added to `PlanParserSuite`
```
sbt> project catalyst
sbt> testOnly *PlanParserSuite
[info] - case insensitive (565 milliseconds)
[info] - explain (9 milliseconds)
[info] - set operations (41 milliseconds)
[info] - common table expressions (31 milliseconds)
[info] - simple select query (47 milliseconds)
[info] - hive-style single-FROM statement (11 milliseconds)
[info] - multi select query (32 milliseconds)
[info] - query organization (41 milliseconds)
[info] - insert into (12 milliseconds)
[info] - aggregation (24 milliseconds)
[info] - limit (11 milliseconds)
[info] - window spec (11 milliseconds)
[info] - lateral view (17 milliseconds)
[info] - joins (62 milliseconds)
[info] - sampled relations (11 milliseconds)
[info] - sub-query (11 milliseconds)
[info] - scalar sub-query (9 milliseconds)
[info] - table reference (2 milliseconds)
[info] - table valued function (8 milliseconds)
[info] - SPARK-20311 range(N) as alias (2 milliseconds)
[info] - SPARK-20841 Support table column aliases in FROM clause (3 milliseconds)
[info] - SPARK-20962 Support subquery column aliases in FROM clause (4 milliseconds)
[info] - SPARK-20963 Support aliases for join relations in FROM clause (3 milliseconds)
[info] - inline table (23 milliseconds)
[info] - simple select query with !> and !< (5 milliseconds)
[info] - select hint syntax (34 milliseconds)
[info] - SPARK-20854: select hint syntax with expressions (12 milliseconds)
[info] - SPARK-20854: multiple hints (4 milliseconds)
[info] - TRIM function (16 milliseconds)
[info] - OVERLAY function (16 milliseconds)
[info] - precedence of set operations (18 milliseconds)
[info] - create/alter view as insert into table (4 milliseconds)
[info] - Invalid insert constructs in the query (10 milliseconds)
[info] - relation in v2 catalog (3 milliseconds)
[info] - CTE with column alias (2 milliseconds)
[info] - statement containing terminal semicolons (3 milliseconds)
[info] ScalaTest
[info] Run completed in 3 seconds, 129 milliseconds.
[info] Total number of tests run: 36
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 36, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
[info] Passed: Total 36, Failed 0, Errors 0, Passed 36
```

### Current behavior:
#### scala
```scala
scala> val df = sql("select 1")
// df: org.apache.spark.sql.DataFrame = [1: int]
scala> df.show()
// +---+
// |  1|
// +---+
// |  1|
// +---+

scala> val df = sql("select 1;")
// org.apache.spark.sql.catalyst.parser.ParseException:
// extraneous input ';' expecting <EOF>(line 1, pos 8)

// == SQL ==
// select 1;
// --------^^^

//   at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:263)
//   at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:130)
//   at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:52)
//   at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:76)
//   at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:605)
//   at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
//   at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:605)
//   ... 47 elided
```
#### pyspark
```python
df = spark.sql('select 1')
df.show()
#+---+
#|  1|
#+---+
#|  1|
#+---+

df = spark.sql('select 1;')
# Traceback (most recent call last):
#   File "<stdin>", line 1, in <module>
#   File "/Users/ssetegne/spark/python/pyspark/sql/session.py", line 646, in sql
#     return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped)
#   File "/Users/ssetegne/spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in # __call__
#   File "/Users/ssetegne/spark/python/pyspark/sql/utils.py", line 102, in deco
#     raise converted
# pyspark.sql.utils.ParseException:
# extraneous input ';' expecting <EOF>(line 1, pos 8)

# == SQL ==
# select 1;
# --------^^^
```

### Behavior after proposed fix:
#### scala
```scala
scala> val df = sql("select 1")
// df: org.apache.spark.sql.DataFrame = [1: int]
scala> df.show()
// +---+
// |  1|
// +---+
// |  1|
// +---+

scala> val df = sql("select 1;")
// df: org.apache.spark.sql.DataFrame = [1: int]
scala> df.show()
// +---+
// |  1|
// +---+
// |  1|
// +---+
```
#### pyspark
```python
df = spark.sql('select 1')
df.show()
#+---+
#|    1 |
#+---+
#|    1 |
#+---+

df = spark.sql('select 1;')
df.show()
#+---+
#|    1 |
#+---+
#|    1 |
#+---+
```

Closes #27567 from samsetegne/semicolon.

Authored-by: samsetegne <samuelsetegne@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-25 15:00:15 +08:00
Kousuke Saruta 999c9ed10c [SPARK-31081][UI][SQL] Make display of stageId/stageAttemptId/taskId of sql metrics toggleable
### What changes were proposed in this pull request?

This is another solution for `SPARK-31081` and #27849 .
I added a checkbox which can toggle display of stageId/taskid in the SQL's DAG page.
Mainly, I implemented the toggleable texts in boxes with HTML label feature provided by `dagre-d3`.
The additional metrics are enclosed by `<span>` and control the appearance of the text.
But the exception is additional metrics in clusters.
We can use HTML label for cluster but layout will be broken so I choosed normal text label for clusters.
Due to that, this solution contains a little bit tricky code in`spark-sql-viz.js` to manipulate the metric texts and generate DOMs.

### Why are the changes needed?

It makes metrics harder to read after #26843 and user may not interest in extra info(stageId/StageAttemptId/taskId ) when they do not need debug.
#27849 control the appearance by a new configuration property but providing a checkbox is more flexible.

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

Yes.
[Additional metrics shown]
![with-checked](https://user-images.githubusercontent.com/4736016/77244214-0f6cd780-6c56-11ea-9275-a30758dd5339.png)

[Additional metrics hidden]
![without-chedked](https://user-images.githubusercontent.com/4736016/77244219-14ca2200-6c56-11ea-9874-33a466085fce.png)

### How was this patch tested?

Tested manually with a simple DataFrame operation.
* The appearance of additional metrics in the boxes are controlled by the newly added checkbox.
* No error found with JS-debugger.
* Checked/not-checked state is preserved after reloading.

Closes #27927 from sarutak/SPARK-31081.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-03-24 13:37:13 -07:00
yi.wu f6ff7d0cf8 [SPARK-30127][SQL] Support case class parameter for typed Scala UDF
### What changes were proposed in this pull request?

To support  case class parameter for typed Scala UDF, e.g.

```
case class TestData(key: Int, value: String)
val f = (d: TestData) => d.key * d.value.toInt
val myUdf = udf(f)
val df = Seq(("data", TestData(50, "2"))).toDF("col1", "col2")
checkAnswer(df.select(myUdf(Column("col2"))), Row(100) :: Nil)
```

### Why are the changes needed?

Currently, Spark UDF can only work on data types like java.lang.String, o.a.s.sql.Row, Seq[_], etc. This is inconvenient if user want to apply an operation on one column, and the column is struct type. You must access data from a Row object, instead of domain object like Dataset operations. It will be great if UDF can work on types that are supported by Dataset, e.g. case class.

And here's benchmark result of using case class comparing to row:

```scala

// case class:  58ms 65ms 59ms 64ms 61ms
// row:         59ms 64ms 73ms 84ms 69ms
val f1 = (d: TestData) => s"${d.key}, ${d.value}"
val f2 = (r: Row) => s"${r.getInt(0)}, ${r.getString(1)}"
val udf1 = udf(f1)
// set spark.sql.legacy.allowUntypedScalaUDF=true
val udf2 = udf(f2, StringType)

val df = spark.range(100000).selectExpr("cast (id as int) as id")
    .select(struct('id, lit("str")).as("col"))
df.cache().collect()

// warmup to exclude some extra influence
df.select(udf1('col)).write.mode(SaveMode.Overwrite).format("noop").save()
df.select(udf2('col)).write.mode(SaveMode.Overwrite).format("noop").save()

start = System.currentTimeMillis()
df.select(udf1('col)).write.mode(SaveMode.Overwrite).format("noop").save()
println(System.currentTimeMillis() - start)

start = System.currentTimeMillis()
df.select(udf2('col)).write.mode(SaveMode.Overwrite).format("noop").save()
println(System.currentTimeMillis() - start)

```

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

Yes. User now could be able to use typed Scala UDF with case class as input parameter.

### How was this patch tested?

Added unit tests.

Closes #27937 from Ngone51/udf_caseclass_support.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-24 23:03:57 +08:00
Wenchen Fan 1d0f54951e [SPARK-31205][SQL] support string literal as the second argument of date_add/date_sub functions
### What changes were proposed in this pull request?

https://github.com/apache/spark/pull/26412 introduced a behavior change that `date_add`/`date_sub` functions can't accept string and double values in the second parameter. This is reasonable as it's error-prone to cast string/double to int at runtime.

However, using string literals as function arguments is very common in SQL databases. To avoid breaking valid use cases that the string literal is indeed an integer, this PR proposes to add ansi_cast for string literal in date_add/date_sub functions. If the string value is not a valid integer, we fail at query compiling time because of constant folding.

### Why are the changes needed?

avoid breaking changes

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

Yes, now 3.0 can run `date_add('2011-11-11', '1')` like 2.4

### How was this patch tested?

new tests.

Closes #27965 from cloud-fan/string.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-24 12:07:22 +08:00
Maxim Gekk aa3a7429f4 [SPARK-31159][SQL][FOLLOWUP] Move checking of the rebaseDateTime flag out of the loop in VectorizedColumnReader
### What changes were proposed in this pull request?
In the PR, I propose to refactor reading of timestamps of the `TIMESTAMP_MILLIS` logical type from Parquet files in `VectorizedColumnReader`, and move checking of the `rebaseDateTime` flag out of the internal loop.

### Why are the changes needed?
To avoid any additional overhead of the checking the SQL config `spark.sql.legacy.parquet.rebaseDateTime.enabled` introduced by the PR https://github.com/apache/spark/pull/27915.

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

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

Closes #27973 from MaxGekk/rebase-parquet-datetime-followup.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 23:02:48 +09:00
LantaoJin 929b794e25
[SPARK-30494][SQL] Fix cached data leakage during replacing an existing view
### What changes were proposed in this pull request?

The cached RDD for plan "select 1" stays in memory forever until the session close. This cached data cannot be used since the view temp1 has been replaced by another plan. It's a memory leak.

We can reproduce by below commands:
```
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

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

scala> spark.sql("create or replace temporary view temp1 as select 1")
scala> spark.sql("cache table temp1")
scala> spark.sql("create or replace temporary view temp1 as select 1, 2")
scala> spark.sql("cache table temp1")
scala> assert(spark.sharedState.cacheManager.lookupCachedData(sql("select 1, 2")).isDefined)
scala> assert(spark.sharedState.cacheManager.lookupCachedData(sql("select 1")).isDefined)
```

### Why are the changes needed?
Fix the memory leak, specially for long running mode.

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

### How was this patch tested?
Add an unit test.

Closes #27185 from LantaoJin/SPARK-30494.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-22 22:22:13 -07:00
Kent Yao 88ae6c4481 [SPARK-31189][SQL][DOCS] Fix errors and missing parts for datetime pattern document
### What changes were proposed in this pull request?

Fix errors and missing parts for datetime pattern document
1. The pattern we use is similar to DateTimeFormatter and SimpleDateFormat but not identical. So we shouldn't use any of them in the API docs but use a link to the doc of our own.
2. Some pattern letters are missing
3. Some pattern letters are explicitly banned - Set('A', 'c', 'e', 'n', 'N')
4. the second fraction pattern different logic for parsing and formatting

### Why are the changes needed?

fix and improve doc
### Does this PR introduce any user-facing change?

yes, new and updated doc
### How was this patch tested?

pass Jenkins
viewed locally with `jekyll serve`
![image](https://user-images.githubusercontent.com/8326978/77044447-6bd3bb00-69fa-11ea-8d6f-7084166c5dea.png)

Closes #27956 from yaooqinn/SPARK-31189.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-20 21:59:26 +08:00
Dongjoon Hyun f1cc86792f [SPARK-31181][SQL][TESTS] Remove the default value assumption on CREATE TABLE test cases
### What changes were proposed in this pull request?

A few `CREATE TABLE` test cases have some assumption on the default value of `LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED`. This PR (SPARK-31181) makes the test cases more explicit from test-case side.

The configuration change was tested via https://github.com/apache/spark/pull/27894 during discussing SPARK-31136. This PR has only the test case part from that PR.

### Why are the changes needed?

This makes our test case more robust in terms of the default value of `LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED`. Even in the case where we switch the conf value, that will be one-liner with no test case changes.

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

No.

### How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #27946 from dongjoon-hyun/SPARK-EXPLICIT-TEST.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-20 12:28:57 +08:00
Takeshi Yamamuro ca499e9409
[SPARK-25121][SQL] Supports multi-part table names for broadcast hint resolution
### What changes were proposed in this pull request?

This pr fixed code to respect a database name for broadcast table hint resolution.
Currently, spark ignores a database name in multi-part names;
```
scala> sql("CREATE DATABASE testDb")
scala> spark.range(10).write.saveAsTable("testDb.t")

// without this patch
scala> spark.range(10).join(spark.table("testDb.t"), "id").hint("broadcast", "testDb.t").explain
== Physical Plan ==
*(2) Project [id#24L]
+- *(2) BroadcastHashJoin [id#24L], [id#26L], Inner, BuildLeft
   :- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false]))
   :  +- *(1) Range (0, 10, step=1, splits=4)
   +- *(2) Project [id#26L]
      +- *(2) Filter isnotnull(id#26L)
         +- *(2) FileScan parquet testdb.t[id#26L] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/Users/maropu/Repositories/spark/spark-2.3.1-bin-hadoop2.7/spark-warehouse..., PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>

// with this patch
scala> spark.range(10).join(spark.table("testDb.t"), "id").hint("broadcast", "testDb.t").explain
== Physical Plan ==
*(2) Project [id#3L]
+- *(2) BroadcastHashJoin [id#3L], [id#5L], Inner, BuildRight
   :- *(2) Range (0, 10, step=1, splits=4)
   +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true]))
      +- *(1) Project [id#5L]
         +- *(1) Filter isnotnull(id#5L)
            +- *(1) FileScan parquet testdb.t[id#5L] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/Users/maropu/Repositories/spark/spark-master/spark-warehouse/testdb.db/t], PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
```

This PR comes from https://github.com/apache/spark/pull/22198

### Why are the changes needed?

For better usability.

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

No.

### How was this patch tested?

Added unit tests.

Closes #27935 from maropu/SPARK-25121-2.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-19 20:11:04 -07:00
Dongjoon Hyun c6a6d5e006
Revert "[SPARK-31170][SQL] Spark SQL Cli should respect hive-site.xml and spark.sql.warehouse.dir"
This reverts commit 5bc0d76591.

Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-19 16:08:51 -07:00
Kris Mok a1776288f4 [SPARK-31187][SQL] Sort the whole-stage codegen debug output by codegenStageId
### What changes were proposed in this pull request?

Spark SQL's whole-stage codegen (WSCG) supports dumping the generated code to help with debugging. One way to get the generated code is through `df.queryExecution.debug.codegen`, or SQL `EXPLAIN CODEGEN` statement.

The generated code is currently printed without specific ordering, which can make debugging a bit annoying. This PR makes a minor improvement to sort the codegen dump by the `codegenStageId`, ascending.

After this change, the following query:
```scala
spark.range(10).agg(sum('id)).queryExecution.debug.codegen
```
will always dump the generated code in a natural, stable order. A version of this example with shorter output is:
```
spark.range(10).agg(sum('id)).queryExecution.debug.codegenToSeq.map(_._1).foreach(println)
*(1) HashAggregate(keys=[], functions=[partial_sum(id#8L)], output=[sum#15L])
+- *(1) Range (0, 10, step=1, splits=16)

*(2) HashAggregate(keys=[], functions=[sum(id#8L)], output=[sum(id)#12L])
+- Exchange SinglePartition, true, [id=#30]
   +- *(1) HashAggregate(keys=[], functions=[partial_sum(id#8L)], output=[sum#15L])
      +- *(1) Range (0, 10, step=1, splits=16)
```

The number of codegen stages within a single SQL query tends to be very small, most likely < 50, so the overhead of adding the sorting shouldn't be significant.

### Why are the changes needed?

Minor improvement to aid WSCG debugging.

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

No user-facing change for end-users; minor change for developers who debug WSCG generated code.

### How was this patch tested?

Manually tested the output; all other tests still pass.

Closes #27955 from rednaxelafx/codegen.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-03-19 20:53:01 +09:00
Maxim Gekk bb295d80e3 [SPARK-31159][SQL] Rebase date/timestamp from/to Julian calendar in parquet
### What changes were proposed in this pull request?
The PR addresses the issue of compatibility with Spark 2.4 and earlier version in reading/writing dates and timestamp via Parquet datasource. Previous releases are based on a hybrid calendar - Julian + Gregorian. Since Spark 3.0, Proleptic Gregorian calendar is used by default, see SPARK-26651. In particular, the issue pops up for dates/timestamps before 1582-10-15 when the hybrid calendar switches from/to Gregorian to/from Julian calendar. The same local date in different calendar is converted to different number of days since the epoch 1970-01-01. For example, the 1001-01-01 date is converted to:
- -719164 in Julian calendar. Spark 2.4 saves the number as a value of DATE type into parquet.
- -719162 in Proleptic Gregorian calendar. Spark 3.0 saves the number as a date value.

According to the parquet spec, parquet timestamps of the `TIMESTAMP_MILLIS`, `TIMESTAMP_MICROS` output type and parquet dates should be based on Proleptic Gregorian calendar but the `INT96` timestamps should be stored as Julian days. Since the version 3.0, Spark conforms the spec but for the backward compatibility with previous version, the PR proposes rebasing from/to Proleptic Gregorian calendar to the hybrid one under the SQL config:
```
spark.sql.legacy.parquet.rebaseDateTime.enabled
```
which is set to `false` by default which means the rebasing is not performed by default.

The details of the implementation:
1. Added 2 methods to `DateTimeUtils` for rebasing microseconds. `rebaseGregorianToJulianMicros()` builds a local timestamp in Proleptic Gregorian calendar, extracts date-time fields `year`, `month`, ..., `second fraction` from the local timestamp and uses them to build another local timestamp based on the hybrid calendar (using `java.util.Calendar` API). After that it calculates the number of microseconds since the epoch using the resulted local timestamp. The function performs the conversion via the system JVM time zone for compatibility with Spark 2.4 and earlier versions. The `rebaseJulianToGregorianMicros()` function does reverse conversion.
2. Added 2 methods to `DateTimeUtils` for rebasing days. `rebaseGregorianToJulianDays()` builds a local date from the passed number of days since the epoch in Proleptic Gregorian calendar, interprets the resulted date as a local date in the hybrid calendar and gets the number of days since the epoch from the resulted local date. The conversion is performed via the `UTC` time zone because the conversion is independent from time zones, and `UTC` is selected to void round issues of casting days to milliseconds and back. The `rebaseJulianToGregorianDays()` functions does revers conversion.
3. Use `rebaseGregorianToJulianMicros()` and `rebaseGregorianToJulianDays()` while saving timestamps/dates to parquet files if the SQL config is on.
4. Use `rebaseJulianToGregorianMicros()` and `rebaseJulianToGregorianDays()` while loading timestamps/dates from parquet files if the SQL config is on.
5. The SQL config `spark.sql.legacy.parquet.rebaseDateTime.enabled` controls conversions from/to dates, timestamps of `TIMESTAMP_MILLIS`, `TIMESTAMP_MICROS`, see the SQL config `spark.sql.parquet.outputTimestampType`.
6. The rebasing is always performed for `INT96` timestamps, independently from `spark.sql.legacy.parquet.rebaseDateTime.enabled`.
7. Supported the vectorized parquet reader, see the SQL config `spark.sql.parquet.enableVectorizedReader`.

### Why are the changes needed?
- For the backward compatibility with Spark 2.4 and earlier versions. The changes allow users to read dates/timestamps saved by previous version, and get the same result. Also after the changes, users can enable the rebasing in write, and save dates/timestamps that can be loaded correctly by Spark 2.4 and earlier versions.
- It fixes the bug of incorrect saving/loading timestamps of the `INT96` type

### Does this PR introduce any user-facing change?
Yes, the timestamp `1001-01-01 01:02:03.123456` saved by Spark 2.4.5 as `TIMESTAMP_MICROS` is interpreted by Spark 3.0.0-preview2 differently:
```scala
scala> spark.read.parquet("/Users/maxim/tmp/before_1582/2_4_5_ts_micros").show(false)
+--------------------------+
|ts                        |
+--------------------------+
|1001-01-07 11:32:20.123456|
+--------------------------+
```
After the changes:
```scala
scala> spark.conf.set("spark.sql.legacy.parquet.rebaseDateTime.enabled", true)

scala> spark.read.parquet("/Users/maxim/tmp/before_1582/2_4_5_ts_micros").show(false)
+--------------------------+
|ts                        |
+--------------------------+
|1001-01-01 01:02:03.123456|
+--------------------------+
```

### How was this patch tested?
1. Added tests to `ParquetIOSuite` to check rebasing in read for regular reader and vectorized parquet reader. The test reads back parquet files saved by Spark 2.4.5 via:
```shell
$ export TZ="America/Los_Angeles"
```
```scala
scala> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
scala> val df = Seq("1001-01-01").toDF("dateS").select($"dateS".cast("date").as("date"))
df: org.apache.spark.sql.DataFrame = [date: date]
scala> df.write.parquet("/Users/maxim/tmp/before_1582/2_4_5_date")

scala> val df = Seq("1001-01-01 01:02:03.123456").toDF("tsS").select($"tsS".cast("timestamp").as("ts"))
df: org.apache.spark.sql.DataFrame = [ts: timestamp]

scala> spark.conf.set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MICROS")
scala> df.write.parquet("/Users/maxim/tmp/before_1582/2_4_5_ts_micros")

scala> spark.conf.set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MILLIS")
scala> df.write.parquet("/Users/maxim/tmp/before_1582/2_4_5_ts_millis")

scala> spark.conf.set("spark.sql.parquet.outputTimestampType", "INT96")
scala> df.write.parquet("/Users/maxim/tmp/before_1582/2_4_5_ts_int96")
```
2. Manually check the write code path. Save date/timestamps (TIMESTAMP_MICROS, TIMESTAMP_MILLIS, INT96) by Spark 3.1.0-SNAPSHOT (after the changes):
```bash
$ export TZ="America/Los_Angeles"
```
```scala
scala> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
scala> spark.conf.set("spark.sql.legacy.parquet.rebaseDateTime.enabled", true)
scala> spark.conf.set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MICROS")
scala> val df = Seq(("1001-01-01", "1001-01-01 01:02:03.123456")).toDF("dateS", "tsS").select($"dateS".cast("date").as("d"), $"tsS".cast("timestamp").as("ts"))
df: org.apache.spark.sql.DataFrame = [d: date, ts: timestamp]
scala> df.write.parquet("/Users/maxim/tmp/before_1582/3_0_0_micros")
scala> spark.read.parquet("/Users/maxim/tmp/before_1582/3_0_0_micros").show(false)
+----------+--------------------------+
|d         |ts                        |
+----------+--------------------------+
|1001-01-01|1001-01-01 01:02:03.123456|
+----------+--------------------------+
```
Read the saved date/timestamp by Spark 2.4.5:
```scala
scala> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
scala> spark.read.parquet("/Users/maxim/tmp/before_1582/3_0_0_micros").show(false)
+----------+--------------------------+
|d         |ts                        |
+----------+--------------------------+
|1001-01-01|1001-01-01 01:02:03.123456|
+----------+--------------------------+
```

Closes #27915 from MaxGekk/rebase-parquet-datetime.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-19 12:49:51 +08:00
Burak Yavuz 4237251861 [SPARK-31178][SQL] Prevent V2 exec nodes from executing multiple times
### What changes were proposed in this pull request?

This PR prevents the execution of V2 DataSource exec nodes multiple times when `collect()` is called on them. For V1 DataSources, commands would be executed as a RunnableCommand, which would cache the result as part of the `ExecutedCommandExec` node. We extend `V2CommandExec` for all the data writing commands so that they only get executed once as well.

### Why are the changes needed?

Calling `collect()` on a SQL command that inserts data or creates a table gets executed multiple times otherwise.

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

Fixes a bug

### How was this patch tested?

Unit tests

Closes #27941 from brkyvz/doubleInsert.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2020-03-18 18:07:24 -07:00
Wenchen Fan 8643e5d9c5 [SPARK-31171][SQL][FOLLOWUP] update document
### What changes were proposed in this pull request?

A followup of https://github.com/apache/spark/pull/27936 to update document.

### Why are the changes needed?

correct document

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

no

### How was this patch tested?

N/A

Closes #27950 from cloud-fan/null.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-03-19 07:29:31 +09:00
Kent Yao 3d695954e5 [SPARK-31150][SQL][FOLLOWUP] handle ' as escape for text
### What changes were proposed in this pull request?

pattern `''` means literal `'`

```sql
select date_format(to_timestamp("11111904-01-23 15:02:01", 'y-MM-dd HH:mm:ss'), "y-MM-dd HH:mm:ss''SSSSSSSSS");
5377-02-14 06:27:19'000000519
```
0946a9514f missed this case and this pr add it back.

### Why are the changes needed?

bugfix

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

no
### How was this patch tested?

add ut

Closes #27949 from yaooqinn/SPARK-31150-2.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-03-19 07:27:06 +09:00
Kent Yao 57fcc49306 [SPARK-31176][SQL] Remove support for 'e'/'c' as datetime pattern charactar
### What changes were proposed in this pull request?

The meaning of 'u' was day number of the week in SimpleDateFormat, it was changed to year in DateTimeFormatter. Now we keep the old meaning of 'u' by substituting 'u' to 'e' internally and use DateTimeFormatter to parse the pattern string. In DateTimeFormatter, the 'e' and 'c' also represents day-of-week. e.g.

```sql
select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uuuu');
select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uuee');
select date_format(timestamp '2019-10-06', 'yyyy-MM-dd eeee');
```
Because of the substitution, they all goes to `.... eeee` silently. The users may congitive problems of their meanings, so we should mark them as illegal pattern characters to stay the same as before.

This pr move the method `convertIncompatiblePattern` from `DatetimeUtils` to `DateTimeFormatterHelper` object, since it is quite specific for `DateTimeFormatterHelper` class.
And 'e' and 'c' char checking in this method.

Besides,`convertIncompatiblePattern` has a bug that will lose the last `'` if it ends with it, this pr fixes this too. e.g.

```sql
spark-sql> select date_format(timestamp "2019-10-06", "yyyy-MM-dd'S'");
20/03/18 11:19:45 ERROR SparkSQLDriver: Failed in [select date_format(timestamp "2019-10-06", "yyyy-MM-dd'S'")]
java.lang.IllegalArgumentException: Pattern ends with an incomplete string literal: uuuu-MM-dd'S

spark-sql> select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'");
NULL
```
### Why are the changes needed?

avoid vagueness
bug fix

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

no, these are not  exposed yet

### How was this patch tested?

add ut

Closes #27939 from yaooqinn/SPARK-31176.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-18 20:19:50 +08:00
Kent Yao f1d27cdd91 [SPARK-31119][SQL] Add interval value support for extract expression as extract source
### What changes were proposed in this pull request?

```
<extract expression> ::= EXTRACT <left paren> <extract field> FROM <extract source> <right paren>

<extract source> ::= <datetime value expression> | <interval value expression>
```
We now only support datetime values as extract source for `extract` expression but it's alternative function `date_part` supports both datetime and interval.

This pr adds interval value support for `extract` expression as extract source

### Why are the changes needed?

For ANSI compliance and the semantic consistency between extract and `date_part`, we support intervals for extract expressions.

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

yes, in the `extract(abc from xyz)` expression, the `xyz` can be intervals

### How was this patch tested?

add unit tests

Closes #27876 from yaooqinn/SPARK-31119.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-18 12:29:39 +08:00
manuzhang 4e4e08f372
[SPARK-31047][SQL] Improve file listing for ViewFileSystem
### What changes were proposed in this pull request?
Use `listLocatedStatus` when `lnMemoryFileIndex` is listing files from a `ViewFileSystem` which should delegate to that of `DistributedFileSystem`.

### Why are the changes needed?
When `ViewFileSystem` is used to manage several `DistributedFileSystem`, the change will improve performance of file listing, especially when there are many files.

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

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

Closes #27801 from manuzhang/spark-31047.

Authored-by: manuzhang <owenzhang1990@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-17 14:23:28 -07:00
Wenchen Fan dc5ebc2d5b
[SPARK-31171][SQL] size(null) should return null under ansi mode
### What changes were proposed in this pull request?

Make `size(null)` return null under ANSI mode, regardless of the `spark.sql.legacy.sizeOfNull` config.

### Why are the changes needed?

In https://github.com/apache/spark/pull/27834, we change the result of `size(null)` to be -1 to match the 2.4 behavior and avoid breaking changes.

However, it's true that the "return -1" behavior is error-prone when being used with aggregate functions. The current ANSI mode controls a bunch of "better behaviors" like failing on overflow. We don't enable these "better behaviors" by default because they are too breaking. The "return null" behavior of `size(null)` is a good fit of the ANSI mode.

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

No as ANSI mode is off by default.

### How was this patch tested?

new tests

Closes #27936 from cloud-fan/null.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-17 11:48:54 -07:00
Kent Yao 5bc0d76591 [SPARK-31170][SQL] Spark SQL Cli should respect hive-site.xml and spark.sql.warehouse.dir
### What changes were proposed in this pull request?

In Spark CLI, we create a hive `CliSessionState` and it does not load the `hive-site.xml`. So the configurations in `hive-site.xml` will not take effects like other spark-hive integration apps.

Also, the warehouse directory is not correctly picked. If the `default` database does not exist, the `CliSessionState` will create one during the first time it talks to the metastore. The `Location` of the default DB will be neither the value of `spark.sql.warehousr.dir` nor the user-specified value of `hive.metastore.warehourse.dir`, but the default value of `hive.metastore.warehourse.dir `which will always be `/user/hive/warehouse`.

### Why are the changes needed?

fix bug for Spark SQL cli to pick right confs

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

yes, the non-exists default database will be created in the location specified by the users via `spark.sql.warehouse.dir` or `hive.metastore.warehouse.dir`, or the default value of `spark.sql.warehouse.dir` if none of them specified.

### How was this patch tested?

add cli ut

Closes #27933 from yaooqinn/SPARK-31170.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-17 23:03:18 +08:00
Kent Yao 0946a9514f [SPARK-31150][SQL] Parsing seconds fraction with variable length for timestamp
### What changes were proposed in this pull request?
This PR is to support parsing timestamp values with variable length second fraction parts.

e.g. 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]' can parse timestamp with 0~6 digit-length second fraction but fail >=7
```sql
select to_timestamp(v, 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') from values
 ('2019-10-06 10:11:12.'),
 ('2019-10-06 10:11:12.0'),
 ('2019-10-06 10:11:12.1'),
 ('2019-10-06 10:11:12.12'),
 ('2019-10-06 10:11:12.123UTC'),
 ('2019-10-06 10:11:12.1234'),
 ('2019-10-06 10:11:12.12345CST'),
 ('2019-10-06 10:11:12.123456PST') t(v)
2019-10-06 03:11:12.123
2019-10-06 08:11:12.12345
2019-10-06 10:11:12
2019-10-06 10:11:12
2019-10-06 10:11:12.1
2019-10-06 10:11:12.12
2019-10-06 10:11:12.1234
2019-10-06 10:11:12.123456

select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
NULL
```
Since 3.0, we use java 8 time API to parse and format timestamp values. when we create the `DateTimeFormatter`, we use `appendPattern` to create the build first, where the 'S..S' part will be parsed to a fixed-length(= `'S..S'.length`). This fits the formatting part but too strict for the parsing part because the trailing zeros are very likely to be truncated.

### Why are the changes needed?

improve timestamp parsing and more compatible with 2.4.x

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

no, the related changes are newly added
### How was this patch tested?

add uts

Closes #27906 from yaooqinn/SPARK-31150.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-17 21:53:46 +08:00
Takeshi Yamamuro 124b4ce2e6
[MINOR][SQL] Update the DataFrameWriter.bucketBy comment
### What changes were proposed in this pull request?

This PR intends to update the `DataFrameWriter.bucketBy` comment for clearly describing that the bucketBy scheme follows a Spark "specific" one.

I saw the questions about the current bucketing compatibility with Hive in [SPARK-31162](https://issues.apache.org/jira/browse/SPARK-31162?focusedCommentId=17060408&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17060408) and [SPARK-17495](https://issues.apache.org/jira/browse/SPARK-17495?focusedCommentId=17059847&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17059847) from users and IMHO the comment is a bit confusing to users about the compatibility

### Why are the changes needed?

To make users understood smoothly.

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

No.

### How was this patch tested?

N/A

Closes #27930 from maropu/UpdateBucketByComment.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-17 00:52:45 -07:00
Wenchen Fan 30d95356f1 [SPARK-31134][SQL] optimize skew join after shuffle partitions are coalesced
### What changes were proposed in this pull request?

Run the `OptimizeSkewedJoin` rule after the `CoalesceShufflePartitions` rule.

### Why are the changes needed?

Remove duplicated coalescing code in `OptimizeSkewedJoin`.

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

No

### How was this patch tested?

existing tests

Closes #27893 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2020-03-17 00:23:16 -07:00
Zhenhua Wang 1369a973cd [SPARK-31164][SQL] Inconsistent rdd and output partitioning for bucket table when output doesn't contain all bucket columns
### What changes were proposed in this pull request?

For a bucketed table, when deciding output partitioning, if the output doesn't contain all bucket columns, the result is `UnknownPartitioning`. But when generating rdd, current Spark uses `createBucketedReadRDD` because it doesn't check if the output contains all bucket columns. So the rdd and its output partitioning are inconsistent.

### Why are the changes needed?

To fix a bug.

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

No.

### How was this patch tested?

Modified existing tests.

Closes #27924 from wzhfy/inconsistent_rdd_partitioning.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Zhenhua Wang <wzh_zju@163.com>
2020-03-17 14:20:16 +08:00
Wenchen Fan d7b97a1d0d [SPARK-31166][SQL] UNION map<null, null> and other maps should not fail
### What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/27542, `map()` returns `map<null, null>` instead of `map<string, string>`. However, this breaks queries which union `map()` and other maps.

The reason is, `TypeCoercion` rules and `Cast` think it's illegal to cast null type map key to other types, as it makes the key nullable, but it's actually legal. This PR fixes it.

### Why are the changes needed?

To avoid breaking queries.

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

Yes, now some queries that work in 2.x can work in 3.0 as well.

### How was this patch tested?

new test

Closes #27926 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-17 12:01:29 +08:00
zero323 01f20394ac [SPARK-30569][SQL][PYSPARK][SPARKR] Add percentile_approx DSL functions
### What changes were proposed in this pull request?

- Adds following overloaded variants to Scala `o.a.s.sql.functions`:

  - `percentile_approx(e: Column, percentage: Array[Double], accuracy: Long): Column`
  - `percentile_approx(columnName: String, percentage: Array[Double], accuracy: Long): Column`
  - `percentile_approx(e: Column, percentage: Double, accuracy: Long): Column`
  - `percentile_approx(columnName: String, percentage: Double, accuracy: Long): Column`
  - `percentile_approx(e: Column, percentage: Seq[Double], accuracy: Long): Column` (primarily for
Python interop).
  - `percentile_approx(columnName: String, percentage: Seq[Double], accuracy: Long): Column`

- Adds `percentile_approx` to `pyspark.sql.functions`.

- Adds `percentile_approx` function to SparkR.

### Why are the changes needed?

Currently we support `percentile_approx` only in SQL expression. It is inconvenient and makes this function relatively unknown.

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

No.

### How was this patch tested?

New unit tests for SparkR an PySpark.

As for now there are no additional tests in Scala API ‒ `ApproximatePercentile` is well tested and Python (including docstrings) and R tests provide additional tests, so it seems unnecessary.

Closes #27278 from zero323/SPARK-30569.

Lead-authored-by: zero323 <mszymkiewicz@gmail.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-17 10:44:21 +09:00
yi.wu cb26f636b0
[SPARK-31163][SQL] TruncateTableCommand with acl/permission should handle non-existed path
### What changes were proposed in this pull request?

This fix #26956
Wrap try-catch on `fs.getFileStatus(path)` within acl/permission in case of the path doesn't exist.

### Why are the changes needed?

`truncate table` may fail to re-create path in case of interruption or something else. As a result, next time we `truncate table` on the same table with acl/permission, it will fail due to `FileNotFoundException`. And it also brings behavior change compares to previous Spark version, which could still `truncate table` successfully even if the path doesn't exist.

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

No.

### How was this patch tested?

Added UT.

Closes #27923 from Ngone51/fix_truncate.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-16 11:45:25 -07:00
HyukjinKwon 6704103499
[SPARK-31146][SQL] Leverage the helper method for aliasing in built-in SQL expressions
### What changes were proposed in this pull request?

This PR is kind of a followup of #26808. It leverages the helper method for aliasing in built-in SQL expressions to use the alias as its output column name where it's applicable.

- `Expression`, `UnaryMathExpression` and `BinaryMathExpression` search the alias in the tags by default.
- When the naming is different in its implementation, it has to be overwritten for the expression specifically. E.g., `CallMethodViaReflection`, `Remainder`, `CurrentTimestamp`,
`FormatString` and `XPathDouble`.

This PR fixes the aliases of the functions below:

| class                    | alias            |
|--------------------------|------------------|
|`Rand`                    |`random`          |
|`Ceil`                    |`ceiling`         |
|`Remainder`               |`mod`             |
|`Pow`                     |`pow`             |
|`Signum`                  |`sign`            |
|`Chr`                     |`char`            |
|`Length`                  |`char_length`     |
|`Length`                  |`character_length`|
|`FormatString`            |`printf`          |
|`Substring`               |`substr`          |
|`Upper`                   |`ucase`           |
|`XPathDouble`             |`xpath_number`    |
|`DayOfMonth`              |`day`             |
|`CurrentTimestamp`        |`now`             |
|`Size`                    |`cardinality`     |
|`Sha1`                    |`sha`             |
|`CallMethodViaReflection` |`java_method`     |

Note: `EqualTo`, `=` and `==` aliases were excluded because it's unable to leverage this helper method. It should fix the parser.

Note: this PR also excludes some instances such as `ToDegrees`, `ToRadians`, `UnaryMinus` and `UnaryPositive` that needs an explicit name overwritten to make the scope of this PR smaller.

### Why are the changes needed?

To respect expression name.

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

Yes, it will change the output column name.

### How was this patch tested?

Manually tested, and unittests were added.

Closes #27901 from HyukjinKwon/31146.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-16 11:22:34 -07:00
Tae-kyeom, Kim e736c62764
[SPARK-31116][SQL] Fix nested schema case-sensitivity in ParquetRowConverter
### What changes were proposed in this pull request?

This PR (SPARK-31116) add caseSensitive parameter to ParquetRowConverter so that it handle materialize parquet properly with respect to case sensitivity

### Why are the changes needed?

From spark 3.0.0, below statement throws IllegalArgumentException in caseInsensitive mode because of explicit field index searching in ParquetRowConverter. As we already constructed parquet requested schema and catalyst requested schema during schema clipping in ParquetReadSupport, just follow these behavior.

```scala
val path = "/some/temp/path"

spark
  .range(1L)
  .selectExpr("NAMED_STRUCT('lowercase', id, 'camelCase', id + 1) AS StructColumn")
  .write.parquet(path)

val caseInsensitiveSchema = new StructType()
  .add(
    "StructColumn",
    new StructType()
      .add("LowerCase", LongType)
      .add("camelcase", LongType))

spark.read.schema(caseInsensitiveSchema).parquet(path).show()
```

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

No. The changes are only in unreleased branches (`master` and `branch-3.0`).

### How was this patch tested?

Passed new test cases that check parquet column selection with respect to schemas and case sensitivities

Closes #27888 from kimtkyeom/parquet_row_converter_case_sensitivity.

Authored-by: Tae-kyeom, Kim <kimtkyeom@devsisters.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-16 10:31:56 -07:00
jiake 21c02ee5d0 [SPARK-30864][SQL][DOC] add the user guide for Adaptive Query Execution
### What changes were proposed in this pull request?
This PR will add the user guide for AQE and the detailed configurations about the three mainly features in AQE.

### Why are the changes needed?
Add the detailed configurations.

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

### How was this patch tested?
only add doc no need ut.

Closes #27616 from JkSelf/aqeuserguide.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-16 23:33:56 +08:00
LantaoJin 08bdc9c9b2 [SPARK-31068][SQL] Avoid IllegalArgumentException in broadcast exchange
### What changes were proposed in this pull request?
Fix the IllegalArgumentException in broadcast exchange when numRows over 341 million but less than 512 million.

Since the maximum number of keys that `BytesToBytesMap` supports is 1 << 29, and only 70% of the slots can be used before growing in `HashedRelation`, So here the limitation should not be greater equal than 341 million (1 << 29 / 1.5(357913941)) instead of 512 million.

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

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

Closes #27828 from LantaoJin/SPARK-31068.

Lead-authored-by: LantaoJin <jinlantao@gmail.com>
Co-authored-by: Alan Jin <jinlantao@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-03-15 20:20:23 -05:00
Shixiong Zhu 1ddf44dfca
[SPARK-31144][SQL] Wrap Error with QueryExecutionException to notify QueryExecutionListener
### What changes were proposed in this pull request?

This PR manually reverts changes in #25292 and then wraps java.lang.Error with `QueryExecutionException` to notify `QueryExecutionListener` to send it to `QueryExecutionListener.onFailure` which only accepts `Exception`.

The bug fix PR for 2.4 is #27904. It needs a separate PR because the touched codes were changed a lot.

### Why are the changes needed?

Avoid API changes and fix a bug.

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

Yes. Reverting an API change happening in 3.0. QueryExecutionListener APIs will be the same as 2.4.

### How was this patch tested?

The new added test.

Closes #27907 from zsxwing/SPARK-31144.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-13 15:55:29 -07:00
Dale Clarke 2a4fed0443 [SPARK-30654][WEBUI] Bootstrap4 WebUI upgrade
### What changes were proposed in this pull request?
Spark's Web UI is using an older version of Bootstrap (v. 2.3.2) for the portal pages. Bootstrap 2.x was moved to EOL in Aug 2013 and Bootstrap 3.x was moved to EOL in July 2019 (https://github.com/twbs/release). Older versions of Bootstrap are also getting flagged in security scans for various CVEs:

https://snyk.io/vuln/SNYK-JS-BOOTSTRAP-72889
https://snyk.io/vuln/SNYK-JS-BOOTSTRAP-173700
https://snyk.io/vuln/npm:bootstrap:20180529
https://snyk.io/vuln/npm:bootstrap:20160627

I haven't validated each CVE, but it would be nice to resolve any potential issues and get on a supported release.

The bad news is that there have been quite a few changes between Bootstrap 2 and Bootstrap 4. I've tried updating the library, refactoring/tweaking the CSS and JS to maintain a similar appearance and functionality, and testing the UI for functionality and appearance. This is a fairly large change so I'm sure additional testing and fixes will be needed.

### How was this patch tested?
This has been manually tested, but there is a ton of functionality and there are many pages and detail pages so it is very possible bugs introduced from the upgrade were missed. Additional testing and feedback is welcomed. If it appears a whole page was missed let me know and I'll take a pass at addressing that page/section.

Closes #27370 from clarkead/bootstrap4-core-upgrade.

Authored-by: Dale Clarke <a.dale.clarke@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-03-13 15:24:48 -07:00
Kousuke Saruta 680981587d [SPARK-31004][WEBUI][SS] Show message for empty Streaming Queries instead of empty timelines and histograms
### What changes were proposed in this pull request?

`StreamingQueryStatisticsPage` shows a message "No visualization information available because there is no batches" instead of showing empty timelines and histograms for empty streaming queries.

[Before this change applied]
![before-fix-for-empty-streaming-query](https://user-images.githubusercontent.com/4736016/75642391-b32e1d80-5c7e-11ea-9c07-e2f0f1b5b4f9.png)

[After this change applied]
![after-fix-for-empty-streaming-query2](https://user-images.githubusercontent.com/4736016/75694583-1904be80-5cec-11ea-9b13-dc7078775188.png)

### Why are the changes needed?

Empty charts are ugly and a little bit confusing.
It's better to clearly say "No visualization information available".

Also, this change fixes a JS error shown in the capture above.
This error occurs because `drawTimeline` in `streaming-page.js` is called even though `formattedDate` will be `undefined` for empty streaming queries.

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

Yes. screen captures are shown above.

### How was this patch tested?

Manually tested by creating an empty streaming query like as follows.
```
val df = spark.readStream.format("socket").options(Map("host"->"<non-existing hostname>", "port"->"...")).load
df.writeStream.format("console").start
```
This streaming query will fail because of `non-existing hostname` and has no batches.

Closes #27755 from sarutak/fix-for-empty-batches.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-03-13 12:58:49 -07:00
Gabor Somogyi 231e65092f [SPARK-30874][SQL] Support Postgres Kerberos login in JDBC connector
### What changes were proposed in this pull request?
When loading DataFrames from JDBC datasource with Kerberos authentication, remote executors (yarn-client/cluster etc. modes) fail to establish a connection due to lack of Kerberos ticket or ability to generate it.

This is a real issue when trying to ingest data from kerberized data sources (SQL Server, Oracle) in enterprise environment where exposing simple authentication access is not an option due to IT policy issues.

In this PR I've added Postgres support (other supported databases will come in later PRs).

What this PR contains:
* Added `keytab` and `principal` JDBC options
* Added `ConnectionProvider` trait and it's impementations:
  * `BasicConnectionProvider` => unsecure connection
  * `PostgresConnectionProvider` => postgres secure connection
* Added `ConnectionProvider` tests
* Added `PostgresKrbIntegrationSuite` docker integration test
* Created `SecurityUtils` to concentrate re-usable security related functionalities
* Documentation

### Why are the changes needed?
Missing JDBC kerberos support.

### Does this PR introduce any user-facing change?
Yes, 2 additional JDBC options added:
* keytab
* principal

If both provided then Spark does kerberos authentication.

### How was this patch tested?
To demonstrate the functionality with a standalone application I've created this repository: https://github.com/gaborgsomogyi/docker-kerberos

* Additional + existing unit tests
* Additional docker integration test
* Test on cluster manually
* `SKIP_API=1 jekyll build`

Closes #27637 from gaborgsomogyi/SPARK-30874.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@apache.org>
2020-03-12 19:04:35 -07:00
Wenchen Fan b27b3c91f1 [SPARK-31090][SPARK-25457] Revert "IntegralDivide returns data type of the operands"
### What changes were proposed in this pull request?

This reverts commit 47d6e80a2e.

### Why are the changes needed?

There is no standard requiring that `div` must return the type of the operand, and always returning long type looks fine. 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 `div` back to be the same as 2.4.

### How was this patch tested?

N/A

Closes #27835 from cloud-fan/revert2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-13 10:47:36 +09:00
Kent Yao fbc9dc7e9d
[SPARK-31129][SQL][TESTS] Fix IntervalBenchmark and DateTimeBenchmark
### What changes were proposed in this pull request?

This PR aims to recover `IntervalBenchmark` and `DataTimeBenchmark` due to banning intervals as output.

### Why are the changes needed?

This PR recovers the benchmark suite.

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

No.

### How was this patch tested?

Manually, re-run the benchmark.

Closes #27885 from yaooqinn/SPARK-31111-2.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-12 12:59:29 -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
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
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
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
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
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
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
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
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
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