Commit graph

6705 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