Commit graph

7234 commits

Author SHA1 Message Date
Maxim Gekk c1f160e097 [SPARK-30648][SQL] Support filters pushdown in JSON datasource
### What changes were proposed in this pull request?
In the PR, I propose to support pushed down filters in JSON datasource. The reason of pushing a filter up to `JacksonParser` is to apply the filter as soon as all its attributes become available i.e. converted from JSON field values to desired values according to the schema. This allows to skip parsing of the rest of JSON record and conversions of other values if the filter returns `false`. This can improve performance when pushed filters are highly selective and conversion of JSON string fields to desired values are comparably expensive ( for example, the conversion to `TIMESTAMP` values).

The main idea behind of `JsonFilters` is to group pushdown filters by their references, convert the grouped filters to expressions, and then compile to predicates. The predicates are indexed by schema field positions. Each predicate has a state with reference counter to non-set row fields. As soon as the counter reaches `0`, it can be applied to the row because all its dependencies has been set. Before processing new row, predicate's reference counter is reset to total number of predicate references (dependencies in a row).

The common code shared between `CSVFilters` and `JsonFilters` is moved to the `StructFilters` class and its companion object.

### Why are the changes needed?
The changes improve performance on synthetic benchmarks up to **27 times** on JDK 8 and **25** times on JDK 11:
```
OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1044-aws
Intel(R) Xeon(R) CPU E5-2670 v2  2.50GHz
Filters pushdown:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
w/o filters                                       25230          25255          22          0.0      252299.6       1.0X
pushdown disabled                                 25248          25282          33          0.0      252475.6       1.0X
w/ filters                                          905            911           8          0.1        9047.9      27.9X
```

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

### How was this patch tested?
- Added new test suites `JsonFiltersSuite` and `JacksonParserSuite`.
- By new end-to-end and case sensitivity tests in `JsonSuite`.
- By `CSVFiltersSuite`, `UnivocityParserSuite` and `CSVSuite`.
- Re-running `CSVBenchmark` and `JsonBenchmark` using Amazon EC2:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge (spot instance) |
| AMI | ami-06f2f779464715dc5 (ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1) |
| Java | OpenJDK8/11 installed by`sudo add-apt-repository ppa:openjdk-r/ppa` & `sudo apt install openjdk-11-jdk`|

and `./dev/run-benchmarks`:
```python
#!/usr/bin/env python3

import os
from sparktestsupport.shellutils import run_cmd

benchmarks = [
    ['sql/test', 'org.apache.spark.sql.execution.datasources.csv.CSVBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.datasources.json.JsonBenchmark']
]

print('Set SPARK_GENERATE_BENCHMARK_FILES=1')
os.environ['SPARK_GENERATE_BENCHMARK_FILES'] = '1'

for b in benchmarks:
    print("Run benchmark: %s" % b[1])
    run_cmd(['build/sbt', '%s:runMain %s' % (b[0], b[1])])
```

Closes #27366 from MaxGekk/json-filters-pushdown.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-17 00:01:13 +09:00
SaurabhChawla 6be8b935a4 [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables
### What changes were proposed in this pull request?
Spark sql commands are failing on selecting the orc tables
Steps to reproduce
Example 1 -
Prerequisite -  This is the location(/Users/test/tpcds_scale5data/date_dim) for orc data which is generated by the hive.
```
val table = """CREATE TABLE `date_dim` (
  `d_date_sk` INT,
  `d_date_id` STRING,
  `d_date` TIMESTAMP,
  `d_month_seq` INT,
  `d_week_seq` INT,
  `d_quarter_seq` INT,
  `d_year` INT,
  `d_dow` INT,
  `d_moy` INT,
  `d_dom` INT,
  `d_qoy` INT,
  `d_fy_year` INT,
  `d_fy_quarter_seq` INT,
  `d_fy_week_seq` INT,
  `d_day_name` STRING,
  `d_quarter_name` STRING,
  `d_holiday` STRING,
  `d_weekend` STRING,
  `d_following_holiday` STRING,
  `d_first_dom` INT,
  `d_last_dom` INT,
  `d_same_day_ly` INT,
  `d_same_day_lq` INT,
  `d_current_day` STRING,
  `d_current_week` STRING,
  `d_current_month` STRING,
  `d_current_quarter` STRING,
  `d_current_year` STRING)
USING orc
LOCATION '/Users/test/tpcds_scale5data/date_dim'"""

spark.sql(table).collect

val u = """select date_dim.d_date_id from date_dim limit 5"""

spark.sql(u).collect
```
Example 2

```
  val table = """CREATE TABLE `test_orc_data` (
  `_col1` INT,
  `_col2` STRING,
  `_col3` INT)
  USING orc"""

spark.sql(table).collect

spark.sql("insert into test_orc_data values(13, '155', 2020)").collect

val df = """select _col2 from test_orc_data limit 5"""
spark.sql(df).collect

```

Its Failing with below error
```
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2.0 (TID 2, 192.168.0.103, executor driver): java.lang.ArrayIndexOutOfBoundsException: 1
    at org.apache.spark.sql.execution.datasources.orc.OrcColumnarBatchReader.initBatch(OrcColumnarBatchReader.java:156)
    at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat.$anonfun$buildReaderWithPartitionValues$7(OrcFileFormat.scala:258)
    at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:141)
    at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:203)
    at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:116)
    at org.apache.spark.sql.execution.FileSourceScanExec$$anon$1.hasNext(DataSourceScanExec.scala:620)
    at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.columnartorow_nextBatch_0$(Unknown Source)
    at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
    at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
    at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:729)
    at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:343)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:895)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:895)
    at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
    at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:372)
    at org.apache.spark.rdd.RDD.iterator(RDD.scala:336)
    at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
    at org.apache.spark.scheduler.Task.run(Task.scala:133)
    at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:445)
    at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1489)
    at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:448)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
    at java.lang.Thread.run(Thread.java:748)`
```

The reason behind this initBatch is not getting the schema that is needed to find out the column value in OrcFileFormat.scala
```
batchReader.initBatch(
 TypeDescription.fromString(resultSchemaString)
```

### Why are the changes needed?
Spark sql queries for orc tables are failing

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

### How was this patch tested?
Unit test is added for this .Also Tested through spark shell and spark submit the failing queries

Closes #29045 from SaurabhChawla100/SPARK-32234.

Lead-authored-by: SaurabhChawla <saurabhc@qubole.com>
Co-authored-by: SaurabhChawla <s.saurabhtim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-16 13:11:47 +00:00
Kent Yao bdeb626c5a [SPARK-32272][SQL] Add SQL standard command SET TIME ZONE
### What changes were proposed in this pull request?

This PR adds the SQL standard command - `SET TIME ZONE` to the current default time zone displacement for the current SQL-session, which is the same as the existing `set spark.sql.session.timeZone=xxx'.

All in all, this PR adds syntax as following,

```
SET TIME ZONE LOCAL;
SET TIME ZONE 'valid time zone';  -- zone offset or region
SET TIME ZONE INTERVAL XXXX; -- xxx must in [-18, + 18] hours, * this range is bigger than ansi  [-14, + 14]
```

### Why are the changes needed?

ANSI compliance and supply pure SQL users a way to retrieve all supported TimeZones

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

yes, add new syntax.

### How was this patch tested?

add unit tests.

and locally verified reference doc

![image](https://user-images.githubusercontent.com/8326978/87510244-c8dc3680-c6a5-11ea-954c-b098be84afee.png)

Closes #29064 from yaooqinn/SPARK-32272.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-16 13:01:53 +00:00
Erik Krogen cf22d947fb [SPARK-32036] Replace references to blacklist/whitelist language with more appropriate terminology, excluding the blacklisting feature
### What changes were proposed in this pull request?

This PR will remove references to these "blacklist" and "whitelist" terms besides the blacklisting feature as a whole, which can be handled in a separate JIRA/PR.

This touches quite a few files, but the changes are straightforward (variable/method/etc. name changes) and most quite self-contained.

### Why are the changes needed?

As per discussion on the Spark dev list, it will be beneficial to remove references to problematic language that can alienate potential community members. One such reference is "blacklist" and "whitelist". While it seems to me that there is some valid debate as to whether these terms have racist origins, the cultural connotations are inescapable in today's world.

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

In the test file `HiveQueryFileTest`, a developer has the ability to specify the system property `spark.hive.whitelist` to specify a list of Hive query files that should be tested. This system property has been renamed to `spark.hive.includelist`. The old property has been kept for compatibility, but will log a warning if used. I am open to feedback from others on whether keeping a deprecated property here is unnecessary given that this is just for developers running tests.

### How was this patch tested?

Existing tests should be suitable since no behavior changes are expected as a result of this PR.

Closes #28874 from xkrogen/xkrogen-SPARK-32036-rename-blacklists.

Authored-by: Erik Krogen <ekrogen@linkedin.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-15 11:40:55 -05:00
Dilip Biswal e4499932da [SPARK-31480][SQL] Improve the EXPLAIN FORMATTED's output for DSV2's Scan Node
### What changes were proposed in this pull request?
Improve the EXPLAIN FORMATTED output of DSV2 Scan nodes (file based ones).

**Before**
```
== Physical Plan ==
* Project (4)
+- * Filter (3)
   +- * ColumnarToRow (2)
      +- BatchScan (1)

(1) BatchScan
Output [2]: [value#7, id#8]
Arguments: [value#7, id#8], ParquetScan(org.apache.spark.sql.test.TestSparkSession17477bbb,Configuration: core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml, hdfs-site.xml, __spark_hadoop_conf__.xml,org.apache.spark.sql.execution.datasources.InMemoryFileIndexa6c363ce,StructType(StructField(value,IntegerType,true)),StructType(StructField(value,IntegerType,true)),StructType(StructField(id,IntegerType,true)),[Lorg.apache.spark.sql.sources.Filter;40fee459,org.apache.spark.sql.util.CaseInsensitiveStringMapfeca1ec6,Vector(isnotnull(id#8), (id#8 > 1)),List(isnotnull(value#7), (value#7 > 2)))
(2) ...
(3) ...
(4) ...
```
**After**
```
== Physical Plan ==
* Project (4)
+- * Filter (3)
   +- * ColumnarToRow (2)
      +- BatchScan (1)

(1) BatchScan
Output [2]: [value#7, id#8]
DataFilters: [isnotnull(value#7), (value#7 > 2)]
Format: parquet
Location: InMemoryFileIndex[....]
PartitionFilters: [isnotnull(id#8), (id#8 > 1)]
PushedFilers: [IsNotNull(id), IsNotNull(value), GreaterThan(id,1), GreaterThan(value,2)]
ReadSchema: struct<value:int>
(2) ...
(3) ...
(4) ...
```
### Why are the changes needed?
The old format is not very readable. This improves the readability of the plan.

### Does this PR introduce any user-facing change?
Yes. the explain output will be different.

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

Closes #28425 from dilipbiswal/dkb_dsv2_explain.

Lead-authored-by: Dilip Biswal <dkbiswal@gmail.com>
Co-authored-by: Dilip Biswal <dkbiswal@apache.org>
Signed-off-by: Dilip Biswal <dkbiswal@apache.org>
2020-07-15 01:28:39 -07:00
Jungtaek Lim (HeartSaVioR) 542aefb4c4 [SPARK-31985][SS] Remove incomplete/undocumented stateful aggregation in continuous mode
### What changes were proposed in this pull request?

This removes the undocumented and incomplete feature of "stateful aggregation" in continuous mode, which would reduce 1100+ lines of code.

### Why are the changes needed?

The work for the feature had been stopped for over an year, and no one asked/requested for the availability of such feature in community. Current state for the feature is that it only works with `coalesce(1)` which force the query to read and process, and write in "a" task, which doesn't make sense in production.

The remaining code increases the work on DSv2 changes as well - that's why I don't simply propose reverting relevant commits - the code path has been changed due to DSv2 evolution.

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

Technically no, because it's never documented and can't be used in production in current shape.

### How was this patch tested?

Existing tests.

Closes #29077 from HeartSaVioR/SPARK-31985.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-07-15 13:40:43 +09:00
HyukjinKwon 03b5707b51 [MINOR][R] Match collectAsArrowToR with non-streaming collectAsArrowToPython
### What changes were proposed in this pull request?

This PR proposes to port forward #29098 to `collectAsArrowToR`. `collectAsArrowToR` follows `collectAsArrowToPython` in branch-2.4 due to the limitation of ARROW-4512. SparkR vectorization currently cannot use streaming format.

### Why are the changes needed?

For simplicity and consistency.

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

No.

### How was this patch tested?

The same code is being tested in `collectAsArrowToPython` of branch-2.4.

Closes #29100 from HyukjinKwon/minor-parts.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-15 08:46:20 +09:00
yi.wu a47b69a88a [SPARK-32307][SQL] ScalaUDF's canonicalized expression should exclude inputEncoders
### What changes were proposed in this pull request?

Override `canonicalized` to empty the `inputEncoders` for the canonicalized `ScalaUDF`.

### Why are the changes needed?

The following fails on `branch-3.0` currently, not on Apache Spark 3.0.0 release.

```scala
spark.udf.register("key", udf((m: Map[String, String]) => m.keys.head.toInt))
Seq(Map("1" -> "one", "2" -> "two")).toDF("a").createOrReplaceTempView("t")
checkAnswer(sql("SELECT key(a) AS k FROM t GROUP BY key(a)"), Row(1) :: Nil)

[info]   org.apache.spark.sql.AnalysisException: expression 't.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
[info] Aggregate [UDF(a#6)], [UDF(a#6) AS k#8]
[info] +- SubqueryAlias t
[info]    +- Project [value#3 AS a#6]
[info]       +- LocalRelation [value#3]
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis(CheckAnalysis.scala:49)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis$(CheckAnalysis.scala:48)
[info]   at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:130)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkValidAggregateExpression$1(CheckAnalysis.scala:257)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$10(CheckAnalysis.scala:259)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$10$adapted(CheckAnalysis.scala:259)
[info]   at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
[info]   at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
[info]   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkValidAggregateExpression$1(CheckAnalysis.scala:259)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$10(CheckAnalysis.scala:259)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$10$adapted(CheckAnalysis.scala:259)
[info]   at scala.collection.immutable.List.foreach(List.scala:392)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkValidAggregateExpression$1(CheckAnalysis.scala:259)
...
```

We use the rule`ResolveEncodersInUDF` to resolve `inputEncoders` and the original`ScalaUDF` instance will be updated to a new `ScalaUDF` instance with the resolved encoders at the end. Note, during encoder resolving, types like `map`, `array` will be resolved to new expression(e.g. `MapObjects`, `CatalystToExternalMap`).

However, `ExpressionEncoder` can't be canonicalized. Thus, the canonicalized `ScalaUDF`s become different even if their original  `ScalaUDF`s are the same. Finally, it fails the `checkValidAggregateExpression` when this `ScalaUDF` is used as a group expression.

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

Yes, users will not hit the exception after this fix.

### How was this patch tested?

Added tests.

Closes #29106 from Ngone51/spark-32307.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-14 12:19:01 -07:00
Sean Owen d6a68e0b67 [SPARK-29292][STREAMING][SQL][BUILD] Get streaming, catalyst, sql compiling for Scala 2.13
### What changes were proposed in this pull request?

Continuation of https://github.com/apache/spark/pull/28971 which lets streaming, catalyst and sql compile for 2.13. Same idea.

### Why are the changes needed?

Eventually, we need to support a Scala 2.13 build, perhaps in Spark 3.1.

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

No.

### How was this patch tested?

Existing tests. (2.13 was not tested; this is about getting it to compile without breaking 2.12)

Closes #29078 from srowen/SPARK-29292.2.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-14 02:06:50 -07:00
angerszhu 5521afbd22 [SPARK-32220][SQL][FOLLOW-UP] SHUFFLE_REPLICATE_NL Hint should not change Non-Cartesian Product join result
### What changes were proposed in this pull request?
follow comment https://github.com/apache/spark/pull/29035#discussion_r453468999
Explain for pr

### Why are the changes needed?
add comment

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

### How was this patch tested?
Not need

Closes #29084 from AngersZhuuuu/follow-spark-32220.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-13 08:23:25 -07:00
angerszhu 6d499647b3 [SPARK-32105][SQL] Refactor current ScriptTransformationExec code
### What changes were proposed in this pull request?

 * Renamed  hive transform scrip class `hive/execution/ScriptTransformationExec` to `hive/execution/HiveScriptTransformationExec` (don't rename file)
 * Extract class `BaseScriptTransformationExec ` about common code used across `SparkScriptTransformationExec(next pr add this)` and `HiveScriptTransformationExec`
 * Extract class `BaseScriptTransformationWriterThread` of writing data thread across `SparkScriptTransformationWriterThread(added next for support transform in sql/core )`  and  `HiveScriptTransformationWriterThread` ,
 * `HiveScriptTransformationWriterThread` additionally supports Hive serde format
 * Rename current `Script` strategies in hive module to `HiveScript`, in next pr will add `SparkScript` strategies for support transform in sql/core.

Todo List;

- Support transform in sql/core base on `BaseScriptTransformationExec`, which would run script operator in SQL mode (without Hive).
The output of script would be read as a string and column values are extracted by using a delimiter (default : tab character)
- For Hive, by default only serde's must be used, and without hive we can run without serde
- Cleanup past hacks that are observed (and people suggest / report), such as
       - [Solve string value error about Date/Timestamp in ScriptTransform](https://issues.apache.org/jira/browse/SPARK-31947)
       - [support use transform with aggregation](https://issues.apache.org/jira/browse/SPARK-28227)
       - [support array/map as transform's input](https://issues.apache.org/jira/browse/SPARK-22435)
- Use code-gen projection to serialize rows to output stream()

### Why are the changes needed?
Support run transform in SQL mode without hive

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

### How was this patch tested?
Added UT

Closes #27983 from AngersZhuuuu/follow_spark_15694.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-13 08:58:25 +00:00
HyukjinKwon c4b0639f83 [SPARK-32270][SQL] Use TextFileFormat in CSV's schema inference with a different encoding
### What changes were proposed in this pull request?

This PR proposes to use text datasource in CSV's schema inference. This shares the same reasons of SPARK-18362, SPARK-19885 and SPARK-19918 - we're currently using Hadoop RDD when the encoding is different, which is unnecessary. This PR completes SPARK-18362, and address the comment at https://github.com/apache/spark/pull/15813#discussion_r90751405.

We should better keep the code paths consistent with existing CSV and JSON datasources as well, but this CSV schema inference with the encoding specified is different from UTF-8 alone.

There can be another story that this PR might lead to a bug fix: Spark session configurations, say Hadoop configurations, are not respected during CSV schema inference when the encoding is different (but it has to be set to Spark context for schema inference when the encoding is different).

### Why are the changes needed?

For consistency, potentially better performance, and fixing a potentially very corner case bug.

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

Virtually no.

### How was this patch tested?

Existing tests should cover.

Closes #29063 from HyukjinKwon/SPARK-32270.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-12 09:44:27 -07:00
yi.wu 004aea8155 [SPARK-32154][SQL] Use ExpressionEncoder for the return type of ScalaUDF to convert to catalyst type
### What changes were proposed in this pull request?

This PR proposes to use `ExpressionEncoder` for the return type of ScalaUDF to convert to the catalyst type, instead of using `CatalystTypeConverters`.

Note, this change only takes effect for typed Scala UDF since its the only case where we know the type tag of the raw type.

### Why are the changes needed?

Users now could register a UDF with `Instant`/`LocalDate` as return types even with `spark.sql.datetime.java8API.enabled=false`. However, the UDF can not really be used.
For example, if we try:

```scala
scala> sql("set spark.sql.datetime.java8API.enabled=false")
scala> spark.udf.register("buildDate", udf{ d: String => java.time.LocalDate.parse(d) })
scala> Seq("2020-07-02").toDF("d").selectExpr("CAST(buildDate(d) AS STRING)").show
```
Then, we will hit the error:
```scala
java.lang.ClassCastException: java.time.LocalDate cannot be cast to java.sql.Date
  at org.apache.spark.sql.catalyst.CatalystTypeConverters$DateConverter$.toCatalystImpl(CatalystTypeConverters.scala:304)
  at org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:107)
  at org.apache.spark.sql.catalyst.CatalystTypeConverters$.$anonfun$createToCatalystConverter$2(CatalystTypeConverters.scala:425)
  at org.apache.spark.sql.catalyst.expressions.ScalaUDF.eval(ScalaUDF.scala:1169)
...
```
as it actually requires enabling `spark.sql.datetime.java8API.enabled` when using the UDF. And I think this could make users get confused.

This happens because when registering the UDF,  Spark actually uses `ExpressionEncoder` to ser/deser types. However, when using UDF, Spark uses `CatalystTypeConverters`, which is under control of `spark.sql.datetime.java8API.enabled`, to ser/deser types. Therefore, Spark would fail to convert the Java8 date time types.

If we could also use `ExpressionEncoder` to ser/deser types for the return type, similar to what we do for the input parameter types, then, UDF could support Instant/LocalDate, event other combined complex types as well.

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

Yes. Before this PR, if users run the demo above, they would hit the error. After this PR, the demo will run successfully.

### How was this patch tested?

Updated 2 tests and added a new one for combined types of `Instant` and `LocalDate`.

Closes #28979 from Ngone51/udf-return-encoder.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-07-12 09:45:18 +09:00
Liang-Chi Hsieh 98504e92a7 [SPARK-29358][SQL] Make unionByName optionally fill missing columns with nulls
### What changes were proposed in this pull request?

This patch proposes to make `unionByName` optionally fill missing columns with nulls.

### Why are the changes needed?

Currently, `unionByName` throws exception if detecting different column names between two Datasets. It is strict requirement and sometimes users require more flexible usage that two Datasets with different subset of columns can be union by name resolution.

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

Yes. Adding overloading `Dataset.unionByName` with a boolean parameter that allows different set of column names between two Datasets. Missing columns at each side, will be filled with null values.

### How was this patch tested?

Unit test.

Closes #28996 from viirya/SPARK-29358.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-11 15:58:40 -07:00
Sean Owen 3ad4863673 [SPARK-29292][SPARK-30010][CORE] Let core compile for Scala 2.13
### What changes were proposed in this pull request?

The purpose of this PR is to partly resolve SPARK-29292, and fully resolve SPARK-30010, which should allow Spark to compile vs Scala 2.13 in Spark Core and up through GraphX (not SQL, Streaming, etc).

Note that we are not trying to determine here whether this makes Spark work on 2.13 yet, just compile, as a prerequisite for assessing test outcomes. However, of course, we need to ensure that the change does not break 2.12.

The changes are, in the main, adding .toSeq and .toMap calls where mutable collections / maps are returned as Seq / Map, which are immutable by default in Scala 2.13. The theory is that it should be a no-op for Scala 2.12 (these return themselves), and required for 2.13.

There are a few non-trivial changes highlighted below.
In particular, to get Core to compile, we need to resolve SPARK-30010 which removes a deprecated SparkConf method

### Why are the changes needed?

Eventually, we need to support a Scala 2.13 build, perhaps in Spark 3.1.

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

Yes, removal of the deprecated SparkConf.setAll overload, which isn't legal in Scala 2.13 anymore.

### How was this patch tested?

Existing tests. (2.13 was not _tested_; this is about getting it to compile without breaking 2.12)

Closes #28971 from srowen/SPARK-29292.1.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-11 14:34:02 -07:00
yi.wu 0c9196e549 [SPARK-32238][SQL] Use Utils.getSimpleName to avoid hitting Malformed class name in ScalaUDF
### What changes were proposed in this pull request?

This PR proposes to use `Utils.getSimpleName(function)` instead of `function.getClass.getSimpleName` to get the class name.

### Why are the changes needed?

For some functions(see the demo below),  using `function.getClass.getSimpleName` can hit "Malformed class name" error.

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

Yes.  For the demo,

```scala
 object MalformedClassObject extends Serializable {
    class MalformedNonPrimitiveFunction extends (String => Int) with Serializable {
      override def apply(v1: String): Int = v1.toInt / 0
    }
  }
 OuterScopes.addOuterScope(MalformedClassObject)
 val f = new MalformedClassObject.MalformedNonPrimitiveFunction()
 Seq("20").toDF("col").select(udf(f).apply(Column("col"))).collect()
```

Before this PR, user can only see the error about "Malformed class name":

```scala
An exception or error caused a run to abort: Malformed class name
java.lang.InternalError: Malformed class name
	at java.lang.Class.getSimpleName(Class.java:1330)
	at org.apache.spark.sql.catalyst.expressions.ScalaUDF.udfErrorMessage$lzycompute(ScalaUDF.scala:1157)
	at org.apache.spark.sql.catalyst.expressions.ScalaUDF.udfErrorMessage(ScalaUDF.scala:1155)
	at org.apache.spark.sql.catalyst.expressions.ScalaUDF.doGenCode(ScalaUDF.scala:1077)
	at org.apache.spark.sql.catalyst.expressions.Expression.$anonfun$genCode$3(Expression.scala:147)
	at scala.Option.getOrElse(Option.scala:189)
	at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:142)
	at org.apache.spark.sql.catalyst.expressions.Alias.genCode(namedExpressions.scala:160)
	at org.apache.spark.sql.execution.ProjectExec.$anonfun$doConsume$1(basicPhysicalOperators.scala:69)
        ...
```

After this PR, user can see the real root cause of the udf failure:

```scala
org.apache.spark.SparkException: Failed to execute user defined function(UDFSuite$MalformedClassObject$MalformedNonPrimitiveFunction: (string) => int)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:753)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:127)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:464)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:467)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.ArithmeticException: / by zero
	at org.apache.spark.sql.UDFSuite$MalformedClassObject$MalformedNonPrimitiveFunction.apply(UDFSuite.scala:677)
	at org.apache.spark.sql.UDFSuite$MalformedClassObject$MalformedNonPrimitiveFunction.apply(UDFSuite.scala:676)
	... 17 more

```
### How was this patch tested?

Added a test.

Closes #29050 from Ngone51/fix-malformed-udf.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-11 06:27:56 -07:00
angerszhu 560fe1f54c [SPARK-32220][SQL] SHUFFLE_REPLICATE_NL Hint should not change Non-Cartesian Product join result
### What changes were proposed in this pull request?
In current Join Hint strategies, if we use SHUFFLE_REPLICATE_NL hint, it will directly convert join to Cartesian Product Join and loss join condition making result not correct.

For Example:
```
spark-sql> select * from test4 order by a asc;
1 2
Time taken: 1.063 seconds, Fetched 4 row(s)20/07/08 14:11:25 INFO SparkSQLCLIDriver: Time taken: 1.063 seconds, Fetched 4 row(s)
spark-sql>select * from test5 order by a asc
1 2
2 2
Time taken: 1.18 seconds, Fetched 24 row(s)20/07/08 14:13:59 INFO SparkSQLCLIDriver: Time taken: 1.18 seconds, Fetched 24 row(s)spar
spark-sql>select /*+ shuffle_replicate_nl(test4) */ * from test4 join test5 where test4.a = test5.a order by test4.a asc ;
1 2 1 2
1 2 2 2
Time taken: 0.351 seconds, Fetched 2 row(s)
20/07/08 14:18:16 INFO SparkSQLCLIDriver: Time taken: 0.351 seconds, Fetched 2 row(s)
```

### Why are the changes needed?
Fix wrong data result

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

### How was this patch tested?
Added UT

Closes #29035 from AngersZhuuuu/SPARK-32220.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-10 09:03:16 -07:00
Kent Yao 4609f1fdab [SPARK-32207][SQL] Support 'F'-suffixed Float Literals
### What changes were proposed in this pull request?

In this PR, I suppose we support 'f'-suffixed float literal, e.g. `select 1.1f`

### Why are the changes needed?

a very common feature across platforms, checked with pg, presto, hive, MySQL...

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

yes,

`select 1.1f` results float value 1.1 instead of throwing AnlaysisExceptiion`Can't extract value from 1: need struct type but got int;`

### How was this patch tested?

add unit tests

Closes #29022 from yaooqinn/SPARK-32207.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-09 19:45:16 -07:00
HyukjinKwon 01e9dd9050 [SPARK-20680][SQL][FOLLOW-UP] Revert NullType.simpleString from 'unknown' to 'null'
### What changes were proposed in this pull request?

This PR proposes to partially reverts the simple string in `NullType` at https://github.com/apache/spark/pull/28833: `NullType.simpleString` back from `unknown` to `null`.

### Why are the changes needed?

- Technically speaking, it's orthogonal with the issue itself, SPARK-20680.
- It needs some more discussion, see https://github.com/apache/spark/pull/28833#issuecomment-655277714

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

It reverts back the user-facing changes at https://github.com/apache/spark/pull/28833.
The simple string of `NullType` is back to `null`.

### How was this patch tested?

I just logically reverted. Jenkins should test it out.

Closes #29041 from HyukjinKwon/SPARK-20680.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-09 19:44:08 -07:00
Dilip Biswal 18aae21d96 [SPARK-31875][SQL] Provide a option to disable user supplied Hints
### What changes were proposed in this pull request?
Introduce a new SQL config `spark.sql.optimizer.ignoreHints`. When this is set to true
application of hints are disabled. This is similar to Oracle's OPTIMIZER_IGNORE_HINTS.
This can be helpful to study the impact of performance difference when hints are applied vs when they are not.

### Why are the changes needed?
Can be helpful to study the impact of performance difference when hints are applied vs when they are not.

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

### How was this patch tested?
New tests added in ResolveHintsSuite.

Closes #28683 from dilipbiswal/disable_hint.

Authored-by: Dilip Biswal <dkbiswal@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-09 18:27:07 -07:00
Erik Erlandson 1cb5bfc47a [SPARK-32159][SQL] Fix integration between Aggregator[Array[_], _, _] and UnresolvedMapObjects
Context: The fix for SPARK-27296 introduced by #25024 allows `Aggregator` objects to appear in queries. This works fine for aggregators with atomic input types, e.g. `Aggregator[Double, _, _]`.

However it can cause a null pointer exception if the input type is `Array[_]`.  This was historically considered an ignorable case for serialization of `UnresolvedMapObjects`, but the new ScalaAggregator class causes these expressions to be serialized over to executors because the resolve-and-bind is being deferred.

### What changes were proposed in this pull request?
A new rule `ResolveEncodersInScalaAgg` that performs the resolution of the expressions contained in the encoders so that properly resolved expressions are serialized over to executors.

### Why are the changes needed?
Applying an aggregator of the form `Aggregator[Array[_], _, _]` using `functions.udaf()` currently causes a null pointer error in Catalyst.

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

### How was this patch tested?
A unit test has been added that does aggregation with array types for input, buffer, and output. I have done additional testing with my own custom aggregators in the spark REPL.

Closes #28983 from erikerlandson/fix-spark-32159.

Authored-by: Erik Erlandson <eerlands@redhat.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-09 08:42:20 +00:00
Jungtaek Lim (HeartSaVioR) 526cb2d1ba [SPARK-32148][SS] Fix stream-stream join issue on missing to copy reused unsafe row
### What changes were proposed in this pull request?

This patch fixes the odd join result being occurred from stream-stream join for state store format V2.

There're some spots on V2 path which leverage UnsafeProjection. As the result row is reused, the row should be copied to avoid changing value during reading (or make sure the caller doesn't affect by such behavior) but `SymmetricHashJoinStateManager.removeByValueCondition` violates the case.

This patch makes `KeyWithIndexToValueRowConverterV2.convertValue` copy the row by itself so that callers don't need to take care about it. This patch doesn't change the behavior of `KeyWithIndexToValueRowConverterV2.convertToValueRow` to avoid double-copying, as the caller is expected to store the row which the implementation of state store will call `copy()`.

This patch adds such behavior into each method doc in `KeyWithIndexToValueRowConverter`, so that further contributors can read through and make sure the change / new addition doesn't break the contract.

### Why are the changes needed?

Stream-stream join with state store format V2 (newly added in Spark 3.0.0) has a serious correctness bug which brings indeterministic result.

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

Yes, some of Spark 3.0.0 users using stream-stream join from the new checkpoint (as the bug exists to only v2 format path) may encounter wrong join result. This patch will fix it.

### How was this patch tested?

Reported case is converted to the new UT, and confirmed UT passed. All UTs in StreamingInnerJoinSuite and StreamingOuterJoinSuite passed as well

Closes #28975 from HeartSaVioR/SPARK-32148.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-09 07:37:06 +00:00
Wenchen Fan 8c5bee599d [SPARK-28067][SPARK-32018] Fix decimal overflow issues
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/27627 to fix the remaining issues. There are 2 issues fixed in this PR:
1. `UnsafeRow.setDecimal` can set an overflowed decimal and causes an error when reading it. The expected behavior is to return null.
2. The update/merge expression for decimal type in `Sum` is wrong. We shouldn't turn the `sum` value back to 0 after it becomes null due to overflow. This issue was hidden because:
2.1 for hash aggregate, the buffer is unsafe row. Due to the first bug, we fail when overflow happens, so there is no chance to mistakenly turn null back to 0.
2.2 for sort-based aggregate, the buffer is generic row. The decimal can overflow (the Decimal class has unlimited precision) and we don't have the null problem.

If we only fix the first bug, then the second bug is exposed and test fails. If we only fix the second bug, there is no way to test it. This PR fixes these 2 bugs together.

### Why are the changes needed?

Fix issues during decimal sum when overflow happens

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

Yes. Now decimal sum can return null correctly for overflow under non-ansi mode.

### How was this patch tested?

new test and updated test

Closes #29026 from cloud-fan/decimal.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-09 15:56:40 +09:00
Takuya UESHIN cfecc2030d [SPARK-32160][CORE][PYSPARK] Disallow to create SparkContext in executors
### What changes were proposed in this pull request?

This PR proposes to disallow to create `SparkContext` in executors, e.g., in UDFs.

### Why are the changes needed?

Currently executors can create SparkContext, but shouldn't be able to create it.

```scala
sc.range(0, 1).foreach { _ =>
  new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
}
```

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

Yes, users won't be able to create `SparkContext` in executors.

### How was this patch tested?

Addes tests.

Closes #28986 from ueshin/issues/SPARK-32160/disallow_spark_context_in_executors.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-09 15:51:56 +09:00
Ryan Blue 3bb1ac597a [SPARK-32168][SQL] Fix hidden partitioning correctness bug in SQL overwrite
### What changes were proposed in this pull request?

When converting an `INSERT OVERWRITE` query to a v2 overwrite plan, Spark attempts to detect when a dynamic overwrite and a static overwrite will produce the same result so it can use the static overwrite. Spark incorrectly detects when dynamic and static overwrites are equivalent when there are hidden partitions, such as `days(ts)`.

This updates the analyzer rule `ResolveInsertInto` to always use a dynamic overwrite when the mode is dynamic, and static when the mode is static. This avoids the problem by not trying to determine whether the two plans are equivalent and always using the one that corresponds to the partition overwrite mode.

### Why are the changes needed?

This is a correctness bug. If a table has hidden partitions, all of the values for those partitions are dropped instead of dynamically overwriting changed partitions.

This only affects SQL commands (not `DataFrameWriter`) writing to tables that have hidden partitions. It is also only a problem when the partition overwrite mode is dynamic.

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

Yes, it fixes the correctness bug detailed above.

### How was this patch tested?

* This updates the in-memory table to support a hidden partition transform, `days`, and adds a test case to `DataSourceV2SQLSuite` in which the table uses this hidden partition function. This test fails without the fix to `ResolveInsertInto`.
* This updates the test case `InsertInto: overwrite - multiple static partitions - dynamic mode` in `InsertIntoTests`. The result of the SQL command is unchanged, but the SQL command will now use a dynamic overwrite so the test now uses `dynamicOverwriteTest`.

Closes #28993 from rdblue/fix-insert-overwrite-v2-conversion.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-08 16:06:40 -07:00
Kousuke Saruta 371b35d2e0 [SPARK-32214][SQL] The type conversion function generated in makeFromJava for "other" type uses a wrong variable
### What changes were proposed in this pull request?

This PR fixes an inconsistency in `EvaluatePython.makeFromJava`, which creates a type conversion function for some Java/Scala types.

`other` is a type but it should actually pass `obj`:

```scala
case other => (obj: Any) => nullSafeConvert(other)(PartialFunction.empty)
```

This does not change the output because it always returns `null` for unsupported datatypes.

### Why are the changes needed?

To make the codes coherent, and consistent.

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

No.

### How was this patch tested?

No behaviour change.

Closes #29029 from sarutak/fix-makeFromJava.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-08 17:46:25 +09:00
LantaoJin b5297c43b0 [SPARK-20680][SQL] Spark-sql do not support for creating table with void column datatype
### What changes were proposed in this pull request?

This is the new PR which to address the close one #17953

1. support "void" primitive data type in the `AstBuilder`, point it to `NullType`
2. forbid creating tables with VOID/NULL column type

### Why are the changes needed?

1. Spark is incompatible with hive void type. When Hive table schema contains void type, DESC table will throw an exception in Spark.

>hive> create table bad as select 1 x, null z from dual;
>hive> describe bad;
OK
x	int
z	void

In Spark2.0.x, the behaviour to read this view is normal:
>spark-sql> describe bad;
x       int     NULL
z       void    NULL
Time taken: 4.431 seconds, Fetched 2 row(s)

But in lastest Spark version, it failed with SparkException: Cannot recognize hive type string: void

>spark-sql> describe bad;
17/05/09 03:12:08 ERROR thriftserver.SparkSQLDriver: Failed in [describe bad]
org.apache.spark.SparkException: Cannot recognize hive type string: void
Caused by: org.apache.spark.sql.catalyst.parser.ParseException:
DataType void() is not supported.(line 1, pos 0)
== SQL ==
void
^^^
        ... 61 more
org.apache.spark.SparkException: Cannot recognize hive type string: void

2. Hive CTAS statements throws error when select clause has NULL/VOID type column since HIVE-11217
In Spark, creating table with a VOID/NULL column should throw readable exception message, include

- create data source table (using parquet, json, ...)
- create hive table (with or without stored as)
- CTAS

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

No

### How was this patch tested?

Add unit tests

Closes #28833 from LantaoJin/SPARK-20680_COPY.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-07 18:58:01 -07:00
Liang-Chi Hsieh 90b9099064 [SPARK-32163][SQL] Nested pruning should work even with cosmetic variations
### What changes were proposed in this pull request?

This patch proposes to deal with cosmetic variations when processing nested column extractors in `NestedColumnAliasing`. Currently if cosmetic variations are in the nested column extractors, the query is not optimized.

### Why are the changes needed?

If the expressions extracting nested fields have cosmetic variations like qualifier difference, currently nested column pruning cannot work well.

For example, two attributes which are semantically the same, are referred in a query, but the nested column extractors of them are treated differently when we deal with nested column pruning.

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

Yes, fixing a bug in nested column pruning.

### How was this patch tested?

Unit test.

Closes #28988 from viirya/SPARK-32163.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-07 11:17:53 -07:00
fqaiser94@gmail.com 4bbc343a4c [SPARK-31317][SQL] Add withField method to Column
### What changes were proposed in this pull request?

Added a new `withField` method to the `Column` class. This method should allow users to add or replace a `StructField` in a `StructType` column (with very similar semantics to the `withColumn` method on `Dataset`).

### Why are the changes needed?

Often Spark users have to work with deeply nested data e.g. to fix a data quality issue with an existing `StructField`. To do this with the existing Spark APIs, users have to rebuild the entire struct column.

For example, let's say you have the following deeply nested data structure which has a data quality issue (`5` is missing):
```
import org.apache.spark.sql._
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._

val data = spark.createDataFrame(sc.parallelize(
      Seq(Row(Row(Row(1, 2, 3), Row(Row(4, null, 6), Row(7, 8, 9), Row(10, 11, 12)), Row(13, 14, 15))))),
      StructType(Seq(
        StructField("a", StructType(Seq(
          StructField("a", StructType(Seq(
            StructField("a", IntegerType),
            StructField("b", IntegerType),
            StructField("c", IntegerType)))),
          StructField("b", StructType(Seq(
            StructField("a", StructType(Seq(
              StructField("a", IntegerType),
              StructField("b", IntegerType),
              StructField("c", IntegerType)))),
            StructField("b", StructType(Seq(
              StructField("a", IntegerType),
              StructField("b", IntegerType),
              StructField("c", IntegerType)))),
            StructField("c", StructType(Seq(
              StructField("a", IntegerType),
              StructField("b", IntegerType),
              StructField("c", IntegerType))))
          ))),
          StructField("c", StructType(Seq(
            StructField("a", IntegerType),
            StructField("b", IntegerType),
            StructField("c", IntegerType))))
        )))))).cache

data.show(false)
+---------------------------------+
|a                                |
+---------------------------------+
|[[1, 2, 3], [[4,, 6], [7, 8, 9]]]|
+---------------------------------+
```
Currently, to replace the missing value users would have to do something like this:
```
val result = data.withColumn("a",
  struct(
    $"a.a",
    struct(
      struct(
        $"a.b.a.a",
        lit(5).as("b"),
        $"a.b.a.c"
      ).as("a"),
      $"a.b.b",
      $"a.b.c"
    ).as("b"),
    $"a.c"
  ))

result.show(false)
+---------------------------------------------------------------+
|a                                                              |
+---------------------------------------------------------------+
|[[1, 2, 3], [[4, 5, 6], [7, 8, 9], [10, 11, 12]], [13, 14, 15]]|
+---------------------------------------------------------------+
```
As you can see above, with the existing methods users must call the `struct` function and list all fields, including fields they don't want to change. This is not ideal as:
>this leads to complex, fragile code that cannot survive schema evolution.
[SPARK-16483](https://issues.apache.org/jira/browse/SPARK-16483)

In contrast, with the method added in this PR, a user could simply do something like this:
```
val result = data.withColumn("a", 'a.withField("b.a.b", lit(5)))
result.show(false)
+---------------------------------------------------------------+
|a                                                              |
+---------------------------------------------------------------+
|[[1, 2, 3], [[4, 5, 6], [7, 8, 9], [10, 11, 12]], [13, 14, 15]]|
+---------------------------------------------------------------+

```

This is the first of maybe a few methods that could be added to the `Column` class to make it easier to manipulate nested data. Other methods under discussion in [SPARK-22231](https://issues.apache.org/jira/browse/SPARK-22231) include `drop` and `renameField`. However, these should be added in a separate PR.

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

No.

### How was this patch tested?

New unit tests were added. Jenkins must pass them.

### Related JIRAs:
- https://issues.apache.org/jira/browse/SPARK-22231
- https://issues.apache.org/jira/browse/SPARK-16483

Closes #27066 from fqaiser94/SPARK-22231-withField.

Lead-authored-by: fqaiser94@gmail.com <fqaiser94@gmail.com>
Co-authored-by: fqaiser94 <fqaiser94@gmail.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-07 16:34:03 +00:00
Wenchen Fan 5d296ed39e [SPARK-32167][SQL] Fix GetArrayStructFields to respect inner field's nullability together
### What changes were proposed in this pull request?

Fix nullability of `GetArrayStructFields`. It should consider both the original array's `containsNull` and the inner field's nullability.

### Why are the changes needed?

Fix a correctness issue.

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

Yes. See the added test.

### How was this patch tested?

a new UT and end-to-end test

Closes #28992 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-06 20:07:33 -07:00
Max Gekk 42f01e314b [SPARK-32130][SQL][FOLLOWUP] Enable timestamps inference in JsonBenchmark
### What changes were proposed in this pull request?
Set the JSON option `inferTimestamp` to `true` for the cases that measure perf of timestamp inference.

### Why are the changes needed?
The PR https://github.com/apache/spark/pull/28966 disabled timestamp inference by default. As a consequence, some benchmarks don't measure perf of timestamp inference from JSON fields. This PR explicitly enable such inference.

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

### How was this patch tested?
By re-generating results of `JsonBenchmark`.

Closes #28981 from MaxGekk/json-inferTimestamps-disable-by-default-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-02 13:26:57 -07:00
stczwd f082a7996a [SPARK-31100][SQL] Check namespace existens when setting namespace
## What changes were proposed in this pull request?
Check the namespace existence while calling "use namespace", and throw NoSuchNamespaceException if namespace not exists.

### Why are the changes needed?
Users need to know that the namespace does not exist when they try to set a wrong namespace.

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

### How was this patch tested?
Run all suites and add a test for this

Closes #27900 from stczwd/SPARK-31100.

Authored-by: stczwd <qcsd2011@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-02 14:49:40 +00:00
Wenchen Fan f83415629b [MINOR][TEST][SQL] Make in-limit.sql more robust
### What changes were proposed in this pull request?

For queries like `t1d in (SELECT t2d FROM  t2 ORDER  BY t2c LIMIT 2)`, the result can be non-deterministic as the result of the subquery may output different results (it's not sorted by `t2d` and it has shuffle).

This PR makes the test more robust by sorting the output column.

### Why are the changes needed?

avoid flaky test

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

no

### How was this patch tested?

N/A

Closes #28976 from cloud-fan/small.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-02 21:04:26 +09:00
Liang-Chi Hsieh 3f7780d30d [SPARK-32136][SQL] NormalizeFloatingNumbers should work on null struct
### What changes were proposed in this pull request?

This patch fixes wrong groupBy result if the grouping key is a null-value struct.

### Why are the changes needed?

`NormalizeFloatingNumbers` reconstructs a struct if input expression is StructType. If the input struct is null, it will reconstruct a struct with null-value fields, instead of null.

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

Yes, fixing incorrect groupBy result.

### How was this patch tested?

Unit test.

Closes #28962 from viirya/SPARK-32136.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-02 13:56:43 +09:00
Liang-Chi Hsieh 2a52a1b300 [SPARK-32056][SQL][FOLLOW-UP] Coalesce partitions for repartiotion hint and sql when AQE is enabled
### What changes were proposed in this pull request?

As the followup of #28900, this patch extends coalescing partitions to repartitioning using hints and SQL syntax without specifying number of partitions, when AQE is enabled.

### Why are the changes needed?

When repartitionning using hints and SQL syntax, we should follow the shuffling behavior of repartition by expression/range to coalesce partitions when AQE is enabled.

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

Yes. After this change, if users don't specify the number of partitions when repartitioning using `REPARTITION`/`REPARTITION_BY_RANGE` hint or `DISTRIBUTE BY`/`CLUSTER BY`, AQE will coalesce partitions.

### How was this patch tested?

Unit tests.

Closes #28952 from viirya/SPARK-32056-sql.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-01 16:14:51 -07:00
Max Gekk bcf23307f4 [SPARK-32130][SQL] Disable the JSON option inferTimestamp by default
### What changes were proposed in this pull request?
Set the JSON option `inferTimestamp` to `false` if an user don't pass it as datasource option.

### Why are the changes needed?
To prevent perf regression while inferring schemas from JSON with potential timestamps fields.

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

### How was this patch tested?
- Modified existing tests in `JsonSuite` and `JsonInferSchemaSuite`.
- Regenerated results of `JsonBenchmark` 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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

Closes #28966 from MaxGekk/json-inferTimestamps-disable-by-default.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-01 15:45:39 -07:00
Wenchen Fan 6edb20df83 [SPARK-31935][SQL][FOLLOWUP] Hadoop file system config should be effective in data source options
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/28760 to fix the remaining issues:
1. should consider data source options when refreshing cache by path at the end of `InsertIntoHadoopFsRelationCommand`
2. should consider data source options when inferring schema for file source
3. should consider data source options when getting the qualified path in file source v2.

### Why are the changes needed?

We didn't catch these issues in https://github.com/apache/spark/pull/28760, because the test case is to check error when initializing the file system. If we initialize the file system multiple times during a simple read/write action, the test case actually only test the first time.

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

No

### How was this patch tested?

rewrite the test to make sure the entire data source read/write action can succeed.

Closes #28948 from cloud-fan/fix.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-07-02 06:09:54 +08:00
Wenchen Fan 7dbd90b68d [SPARK-31797][SQL][FOLLOWUP] TIMESTAMP_SECONDS supports fractional input
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/28534 , to make `TIMESTAMP_SECONDS` function support fractional input as well.

### Why are the changes needed?

Previously the cast function can cast fractional values to timestamp. Now we suggest users to ues these new functions, and we need to cover all the cast use cases.

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

Yes, now `TIMESTAMP_SECONDS` function accepts fractional input.

### How was this patch tested?

new tests

Closes #28956 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-01 08:03:46 -07:00
angerszhu 15fb5d7677 [SPARK-28169][SQL] Convert scan predicate condition to CNF
### What changes were proposed in this pull request?
Spark can't push down scan predicate condition of **Or**:
Such as if I have a table `default.test`, it's partition col is `dt`,
If we use query :
```
select * from default.test
where dt=20190625 or (dt = 20190626 and id in (1,2,3) )
```

In this case, Spark will resolve **Or** condition as one expression, and since this expr has reference of "id", then it can't been push down.

Base on pr https://github.com/apache/spark/pull/28733, In my PR ,  for SQL like
`select * from default.test`
 `where  dt = 20190626  or  (dt = 20190627  and xxx="a")   `

For this  condition `dt = 20190626  or  (dt = 20190627  and xxx="a"   )`, it will  been converted  to CNF
```
(dt = 20190626 or dt = 20190627) and (dt = 20190626 or xxx = "a" )
```
then condition `dt = 20190626 or dt = 20190627` will be push down when partition pruning

### Why are the changes needed?
Optimize partition pruning

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

### How was this patch tested?
Added UT

Closes #28805 from AngersZhuuuu/cnf-for-partition-pruning.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-01 12:00:15 +00:00
HyukjinKwon 8194d9ef78 [SPARK-32142][SQL][TESTS] Keep the original tests and codes to avoid potential conflicts in dev
### What changes were proposed in this pull request?

This PR proposes to partially reverts back in the tests and some codes at https://github.com/apache/spark/pull/27728 without touching any behaivours.

Most of changes in tests are back before #27728 by combining `withNestedDataFrame` and `withParquetDataFrame`.

Basically, it addresses the comments https://github.com/apache/spark/pull/27728#discussion_r397655390, and my own comment in another PR at https://github.com/apache/spark/pull/28761#discussion_r446761037

### Why are the changes needed?

For maintenance purpose and to avoid a potential conflicts during backports. And also in case when other codes are matched with this.

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

No, dev-only.

### How was this patch tested?

Manually tested.

Closes #28955 from HyukjinKwon/SPARK-25556-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-01 14:15:02 +09:00
Gabor Somogyi bbd0275dfd [MINOR][SQL] Fix spaces in JDBC connection providers
### What changes were proposed in this pull request?
JDBC connection providers implementation formatted in a wrong way. In this PR I've fixed the formatting.

### Why are the changes needed?
Wrong spacing in JDBC connection providers.

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

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

Closes #28945 from gaborgsomogyi/provider_spacing.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-30 11:18:16 -07:00
Gabor Somogyi 67cb7eaa65 [SPARK-31336][SQL] Support Oracle 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 Oracle support.

What this PR contains:
* Added `OracleConnectionProvider`
* Added `OracleConnectionProviderSuite`

### 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 Oracle using kerberos.

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

Closes #28863 from gaborgsomogyi/SPARK-31336.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-30 10:30:22 -07:00
Jungtaek Lim (HeartSaVioR) 5472170a2b [SPARK-29999][SS][FOLLOWUP] Fix test to check the actual metadata log directory
### What changes were proposed in this pull request?

This patch fixes the missed spot - the test initializes FileStreamSinkLog with its "output" directory instead of "metadata" directory, hence the verification against sink log was no-op.

### Why are the changes needed?

Without the fix, the verification against sink log was no-op.

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

No.

### How was this patch tested?

Checked with debugger in test, and verified `allFiles()` returns non-zero entries. (It returned zero entry, as there's no metadata.)

Closes #28930 from HeartSaVioR/SPARK-29999-FOLLOWUP-fix-test.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-30 08:09:18 +00:00
yi.wu 6fcb70e0ca [SPARK-32055][CORE][SQL] Unify getReader and getReaderForRange in ShuffleManager
### What changes were proposed in this pull request?

This PR tries to unify the method `getReader` and `getReaderForRange` in `ShuffleManager`.

### Why are the changes needed?

Reduce the duplicate codes, simplify the implementation, and for better maintenance.

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

No.

### How was this patch tested?

Covered by existing tests.

Closes #28895 from Ngone51/unify-getreader.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-29 11:37:03 +00:00
Liang-Chi Hsieh 4204a63d4f [SPARK-32056][SQL] Coalesce partitions for repartition by expressions when AQE is enabled
### What changes were proposed in this pull request?

This patch proposes to coalesce partitions for repartition by expressions without specifying number of partitions, when AQE is enabled.

### Why are the changes needed?

When repartition by some partition expressions, users can specify number of partitions or not. If  the number of partitions is specified, we should not coalesce partitions because it breaks user expectation. But if without specifying number of partitions, AQE should be able to coalesce partitions as other shuffling.

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

Yes. After this change, if users don't specify the number of partitions when repartitioning data by expressions, AQE will coalesce partitions.

### How was this patch tested?

Added unit test.

Closes #28900 from viirya/SPARK-32056.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-29 11:33:40 +00:00
Wenchen Fan 835ef425d0 [SPARK-32038][SQL][FOLLOWUP] Make the alias name pretty after float/double normalization
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/28876/files

This PR proposes to use the name of the original expression, as the alias name of the normalization expression.

### Why are the changes needed?

make the query plan looks pretty when EXPLAIN.

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

No

### How was this patch tested?

manually explain the query

Closes #28919 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-28 21:55:19 -07:00
yi.wu 0ec17c989d [SPARK-32090][SQL] Improve UserDefinedType.equal() to make it be symmetrical
### What changes were proposed in this pull request?

This PR fix `UserDefinedType.equal()` by comparing the UDT class instead of checking `acceptsType()`.

### Why are the changes needed?

It's weird that equality comparison between two UDT types can have different result by switching the order:

```scala
// ExampleSubTypeUDT.userClass is a subclass of ExampleBaseTypeUDT.userClass
val udt1 = new ExampleBaseTypeUDT
val udt2 = new ExampleSubTypeUDT
println(udt1 == udt2) // true
println(udt2 == udt1) // false
```

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

Yes.

Before:
```scala
// ExampleSubTypeUDT.userClass is a subclass of ExampleBaseTypeUDT.userClass
val udt1 = new ExampleBaseTypeUDT
val udt2 = new ExampleSubTypeUDT
println(udt1 == udt2) // true
println(udt2 == udt1) // false
```

After:
```scala
// ExampleSubTypeUDT.userClass is a subclass of ExampleBaseTypeUDT.userClass
val udt1 = new ExampleBaseTypeUDT
val udt2 = new ExampleSubTypeUDT
println(udt1 == udt2) // false
println(udt2 == udt1) // false
```

### How was this patch tested?

Added a unit test.

Closes #28923 from Ngone51/fix-udt-equal.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-28 21:49:10 -07:00
Yuanjian Li f944603872 [SPARK-32126][SS] Scope Session.active in IncrementalExecution
### What changes were proposed in this pull request?

The `optimizedPlan` in IncrementalExecution should also be scoped in `withActive`.

### Why are the changes needed?

Follow-up of SPARK-30798 for the Streaming side.

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

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

Closes #28936 from xuanyuanking/SPARK-30798-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-28 21:35:59 -07:00
Max Gekk 8c44d74463 [SPARK-32071][SQL][TESTS] Add make_interval benchmark
### What changes were proposed in this pull request?
Add benchmarks for interval constructor `make_interval` and measure perf of 4 cases:
1. Constant (year, month)
2. Constant (week, day)
3. Constant (hour, minute, second, second fraction)
4. All fields are NOT constant.

The benchmark results are generated 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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

### Why are the changes needed?
To have a base line for future perf improvements of `make_interval`, and to prevent perf regressions in the future.

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

### How was this patch tested?
By running `IntervalBenchmark` via:
```
$ SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.IntervalBenchmark"
```

Closes #28905 from MaxGekk/benchmark-make_interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-27 17:54:06 -07:00
GuoPhilipse ac3a0551d8 [SPARK-32088][PYTHON] Pin the timezone in timestamp_seconds doctest
### What changes were proposed in this pull request?

Add American timezone during timestamp_seconds doctest

### Why are the changes needed?

`timestamp_seconds` doctest in `functions.py` used default timezone to get expected result
For example:

```python
>>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time'])
>>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect()
[Row(ts=datetime.datetime(2008, 12, 25, 7, 30))]
```

But when we have a non-american timezone, the test case will get different test result.

For example, when we set current timezone as `Asia/Shanghai`, the test result will be

```
[Row(ts=datetime.datetime(2008, 12, 25, 23, 30))]
```

So no matter where we run the test case ,we will always get the expected permanent result if we set the timezone on one specific area.

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

No

### How was this patch tested?

Unit test

Closes #28932 from GuoPhilipse/SPARK-32088-fix-timezone-issue.

Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com>
Co-authored-by: GuoPhilipse <guofei_ok@126.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-26 19:06:31 -07:00
Pablo Langa bbb2cba615 [SPARK-32025][SQL] Csv schema inference problems with different types in the same column
### What changes were proposed in this pull request?

This pull request fixes a bug present in the csv type inference.
We have problems when we have different types in the same column.

**Previously:**
```
$ cat /example/f1.csv
col1
43200000
true

spark.read.csv(path="file:///example/*.csv", header=True, inferSchema=True).show()
+----+
|col1|
+----+
|null|
|true|
+----+

root
 |-- col1: boolean (nullable = true)
```
**Now**
```
spark.read.csv(path="file:///example/*.csv", header=True, inferSchema=True).show()
+-------------+
|col1          |
+-------------+
|43200000 |
|true           |
+-------------+

root
 |-- col1: string (nullable = true)
```

Previously the hierarchy of type inference is the following:

> IntegerType
> > LongType
> > > DecimalType
> > > > DoubleType
> > > > > TimestampType
> > > > > > BooleanType
> > > > > > > StringType

So, when, for example, we have integers in one column, and the last element is a boolean, all the column is inferred as a boolean column incorrectly and all the number are shown as null when you see the data

We need the following hierarchy. When we have different numeric types in the column it will be resolved correctly. And when we have other different types it will be resolved as a String type column
> IntegerType
> > LongType
> > > DecimalType
> > > > DoubleType
> > > > > StringType

> TimestampType
> > StringType

> BooleanType
> > StringType

> StringType

### Why are the changes needed?

Fix the bug explained

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

No

### How was this patch tested?

Unit test and manual tests

Closes #28896 from planga82/feature/SPARK-32025_csv_inference.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-26 10:41:27 +09:00
yi.wu 47fb9d6054 [SPARK-32087][SQL] Allow UserDefinedType to use encoder to deserialize rows in ScalaUDF as well
### What changes were proposed in this pull request?

This PR tries to address the comment: https://github.com/apache/spark/pull/28645#discussion_r442183888
It changes `canUpCast/canCast` to allow cast from sub UDT to base UDT, in order to achieve the goal to allow UserDefinedType to use `ExpressionEncoder` to deserialize rows in ScalaUDF as well.

One thing that needs to mention is, even we allow cast from sub UDT to base UDT, it doesn't really do the cast in `Cast`. Because, yet, sub UDT and base UDT are considered as the same type(because of #16660), see:

5264164a67/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala (L81-L86)

5264164a67/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala (L92-L95)

Therefore, the optimize rule `SimplifyCast` will eliminate the cast at the end.

### Why are the changes needed?

Reduce the special case caused by `UserDefinedType` in `ResolveEncodersInUDF` and `ScalaUDF`.

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

No.

### How was this patch tested?

It should be covered by the test of `SPARK-19311`, which is also updated a little in this PR.

Closes #28920 from Ngone51/fix-udf-udt.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-24 14:50:45 +00:00
ulysses 9f540fac2e [SPARK-32062][SQL] Reset listenerRegistered in SparkSession
### What changes were proposed in this pull request?

Reset listenerRegistered when application end.

### Why are the changes needed?

Within a jvm, stop and create `SparkContext` multi times will cause the bug.

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

No.

### How was this patch tested?

Add UT.

Closes #28899 from ulysses-you/SPARK-32062.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-24 04:50:46 +00:00
Max Gekk 045106e29d [SPARK-32072][CORE][TESTS] Fix table formatting with benchmark results
### What changes were proposed in this pull request?
Set column width w/ benchmark names to maximum of either
1. 40 (before this PR) or
2. The length of benchmark name or
3. Maximum length of cases names

### Why are the changes needed?
To improve readability of benchmark results. For example, `MakeDateTimeBenchmark`.

Before:
```
make_timestamp():                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
prepare make_timestamp()                           3636           3673          38          0.3        3635.7       1.0X
make_timestamp(2019, 1, 2, 3, 4, 50.123456)             94             99           4         10.7          93.8      38.8X
make_timestamp(2019, 1, 2, 3, 4, 60.000000)             68             80          13         14.6          68.3      53.2X
make_timestamp(2019, 12, 31, 23, 59, 60.00)             65             79          19         15.3          65.3      55.7X
make_timestamp(*, *, *, 3, 4, 50.123456)            271            280          14          3.7         270.7      13.4X
```

After:
```
make_timestamp():                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------
prepare make_timestamp()                              3694           3745          82          0.3        3694.0       1.0X
make_timestamp(2019, 1, 2, 3, 4, 50.123456)             82             90           9         12.2          82.3      44.9X
make_timestamp(2019, 1, 2, 3, 4, 60.000000)             72             77           5         13.9          71.9      51.4X
make_timestamp(2019, 12, 31, 23, 59, 60.00)             67             71           5         15.0          66.8      55.3X
make_timestamp(*, *, *, 3, 4, 50.123456)               273            289          14          3.7         273.2      13.5X
```

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

### How was this patch tested?
By re-generating benchmark results for `MakeDateTimeBenchmark`:
```
$ SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.MakeDateTimeBenchmark"
```
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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

Closes #28906 from MaxGekk/benchmark-table-formatting.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-24 04:43:53 +00:00
Max Gekk e00f43cb86 [SPARK-32043][SQL] Replace Decimal by Int op in make_interval and make_timestamp
### What changes were proposed in this pull request?
Replace Decimal by Int op in the `MakeInterval` & `MakeTimestamp` expression. For instance, `(secs * Decimal(MICROS_PER_SECOND)).toLong` can be replaced by the unscaled long because the former one already contains microseconds.

### Why are the changes needed?
To improve performance.

Before:
```
make_timestamp():                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
...
make_timestamp(2019, 1, 2, 3, 4, 50.123456)             94             99           4         10.7          93.8      38.8X
```

After:
```
make_timestamp(2019, 1, 2, 3, 4, 50.123456)             76             92          15         13.1          76.5      48.1X
```

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

### How was this patch tested?
- By existing test suites `IntervalExpressionsSuite`, `DateExpressionsSuite` and etc.
- Re-generate results of `MakeDateTimeBenchmark` 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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

Closes #28886 from MaxGekk/make_interval-opt-decimal.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-23 11:45:12 +00:00
yi.wu 338efee509 [SPARK-32031][SQL] Fix the wrong references of the PartialMerge/Final AggregateExpression
### What changes were proposed in this pull request?

This PR changes the references of the `PartialMerge`/`Final` `AggregateExpression` from `aggBufferAttributes` to `inputAggBufferAttributes`.

After this change, the tests of `SPARK-31620` can fail on the assertion of `QueryTest.assertEmptyMissingInput`.  So, this PR also fixes it by overriding the `inputAggBufferAttributes` of the Aggregate operators.

### Why are the changes needed?

With my understanding of Aggregate framework, especially, according to the logic of `AggUtils.planAggXXX`, I think for the `PartialMerge`/`Final` `AggregateExpression` the right references should be `inputAggBufferAttributes`.

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

No.

### How was this patch tested?

Before this patch, for an Aggregate operator, its input attributes will always be equal to or more than(because it refers to its own attributes while it should refer to the attributes from the child) its reference attributes. Therefore, its missing inputs must always be empty and break nothing. Thus, it's impossible to add a UT for this patch.

However, after correcting the right references in this PR, the problem is then exposed by `QueryTest.assertEmptyMissingInput` in the UT of SPARK-31620, since missing inputs are no longer always empty. This PR can fix the problem.

Closes #28869 from Ngone51/fix-agg-reference.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-22 13:59:46 +00:00
Liang-Chi Hsieh 2e4557f45c [SPARK-32038][SQL] NormalizeFloatingNumbers should also work on distinct aggregate
### What changes were proposed in this pull request?

This patch applies `NormalizeFloatingNumbers` to distinct aggregate to fix a regression of distinct aggregate on NaNs.

### Why are the changes needed?

We added `NormalizeFloatingNumbers` optimization rule in 3.0.0 to normalize special floating numbers (NaN and -0.0). But it is missing in distinct aggregate so causes a regression. We need to apply this rule on distinct aggregate to fix it.

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

Yes, fixing a regression of distinct aggregate on NaNs.

### How was this patch tested?

Added unit test.

Closes #28876 from viirya/SPARK-32038.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-22 04:58:22 -07:00
Yuanjian Li 6fdea63b15 [SPARK-31905][SS] Add compatibility tests for streaming state store format
### What changes were proposed in this pull request?
Add compatibility tests for streaming state store format.

### Why are the changes needed?
After SPARK-31894, we have a validation checking for the streaming state store. It's better to add integrated tests in the PR builder as soon as the breaking changes introduced.

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

### How was this patch tested?
Test only.

Closes #28725 from xuanyuanking/compatibility_check.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-22 07:56:59 +00:00
ulysses 978493467c [SPARK-32019][SQL] Add spark.sql.files.minPartitionNum config
### What changes were proposed in this pull request?

Add a new config `spark.sql.files.minPartitionNum` to control file split partition in local session.

### Why are the changes needed?

Aims to control file split partitions in session level.
More details see discuss in [PR-28778](https://github.com/apache/spark/pull/28778).

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

Yes, new config.

### How was this patch tested?

Add UT.

Closes #28853 from ulysses-you/SPARK-32019.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-20 18:38:44 -07:00
Max Gekk 66ba35666a [SPARK-32021][SQL] Increase precision of seconds and fractions of make_interval
### What changes were proposed in this pull request?
Change precision of seconds and its fraction from 8 to 18 to be able to construct intervals of max allowed microseconds value (long).

### Why are the changes needed?
To improve UX of Spark SQL.

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

### How was this patch tested?
- Add tests to IntervalExpressionsSuite
- Add an example to the `MakeInterval` expression
- Add tests to `interval.sql`

Closes #28873 from MaxGekk/make_interval-sec-precision.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-19 19:33:13 -07:00
Terry Kim 7b8683820b [SPARK-31350][SQL] Coalesce bucketed tables for sort merge join if applicable
### What changes were proposed in this pull request?

When two bucketed tables with different number of buckets are joined, it can introduce a full shuffle:
```
spark.conf.set("spark.sql.autoBroadcastJoinThreshold", "0")
val df1 = (0 until 20).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k")
val df2 = (0 until 20).map(i => (i % 7, i % 11, i.toString)).toDF("i", "j", "k")
df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1")
df2.write.format("parquet").bucketBy(4, "i").saveAsTable("t2")
val t1 = spark.table("t1")
val t2 = spark.table("t2")
val joined = t1.join(t2, t1("i") === t2("i"))
joined.explain

== Physical Plan ==
*(5) SortMergeJoin [i#44], [i#50], Inner
:- *(2) Sort [i#44 ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(i#44, 200), true, [id=#105]
:     +- *(1) Project [i#44, j#45, k#46]
:        +- *(1) Filter isnotnull(i#44)
:           +- *(1) ColumnarToRow
:              +- FileScan parquet default.t1[i#44,j#45,k#46] Batched: true, DataFilters: [isnotnull(i#44)], Format: Parquet, Location: InMemoryFileIndex[...], PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int,j:int,k:string>, SelectedBucketsCount: 8 out of 8
+- *(4) Sort [i#50 ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(i#50, 200), true, [id=#115]
      +- *(3) Project [i#50, j#51, k#52]
         +- *(3) Filter isnotnull(i#50)
            +- *(3) ColumnarToRow
               +- FileScan parquet default.t2[i#50,j#51,k#52] Batched: true, DataFilters: [isnotnull(i#50)], Format: Parquet, Location: InMemoryFileIndex[...], PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int,j:int,k:string>, SelectedBucketsCount: 4 out of 4
```
This PR proposes to introduce coalescing buckets when the following conditions are met to eliminate the full shuffle:
- Join is the sort merge one (which is created only for equi-join).
- Join keys match with output partition expressions on their respective sides.
- The larger bucket number is divisible by the smaller bucket number.
- `spark.sql.bucketing.coalesceBucketsInSortMergeJoin.enabled` is set to `true`.
- The ratio of the number of buckets should be less than the value set in `spark.sql.bucketing.coalesceBucketsInSortMergeJoin.maxBucketRatio`.

### Why are the changes needed?

Eliminating the full shuffle can benefit for scenarios where two large tables are joined. Especially when the tables are already bucketed but differ in the number of buckets, we could take advantage of it.

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

If the bucket coalescing conditions explained above are met, a full shuffle can be eliminated (also note that you will see `SelectedBucketsCount: 8 out of 8 (Coalesced to 4)` in the physical plan):
```
spark.conf.set("spark.sql.autoBroadcastJoinThreshold", "0")
spark.conf.set("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.enabled", "true")
val df1 = (0 until 20).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k")
val df2 = (0 until 20).map(i => (i % 7, i % 11, i.toString)).toDF("i", "j", "k")
df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1")
df2.write.format("parquet").bucketBy(4, "i").saveAsTable("t2")
val t1 = spark.table("t1")
val t2 = spark.table("t2")
val joined = t1.join(t2, t1("i") === t2("i"))
joined.explain

== Physical Plan ==
*(3) SortMergeJoin [i#44], [i#50], Inner
:- *(1) Sort [i#44 ASC NULLS FIRST], false, 0
:  +- *(1) Project [i#44, j#45, k#46]
:     +- *(1) Filter isnotnull(i#44)
:        +- *(1) ColumnarToRow
:           +- FileScan parquet default.t1[i#44,j#45,k#46] Batched: true, DataFilters: [isnotnull(i#44)], Format: Parquet, Location: InMemoryFileIndex[...], PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int,j:int,k:string>, SelectedBucketsCount: 8 out of 8 (Coalesced to 4)
+- *(2) Sort [i#50 ASC NULLS FIRST], false, 0
   +- *(2) Project [i#50, j#51, k#52]
      +- *(2) Filter isnotnull(i#50)
         +- *(2) ColumnarToRow
            +- FileScan parquet default.t2[i#50,j#51,k#52] Batched: true, DataFilters: [isnotnull(i#50)], Format: Parquet, Location: InMemoryFileIndex[...], PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int,j:int,k:string>, SelectedBucketsCount: 4 out of 4
```

### How was this patch tested?

Added unit tests

Closes #28123 from imback82/coalescing_bucket.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-06-20 08:20:45 +09:00
yi.wu 5ee5cfd9c0 [SPARK-31826][SQL] Support composed type of case class for typed Scala UDF
### What changes were proposed in this pull request?

This PR adds support for typed Scala UDF to accept composed type of case class, e.g. Seq[T], Array[T], Map[Int, T] (assuming T is case class type), as input parameter type.

### Why are the changes needed?

After #27937, typed Scala UDF now has supported case class as its input parameter type. However, it can not accept the composed type of case class, such as Seq[T], Array[T], Map[Int, T] (assuming T is case class type), which causing confuse(e.g. https://github.com/apache/spark/pull/27937#discussion_r422699979) to the user.

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

Yes.

Run the query:

```
scala> case class Person(name: String, age: Int)
scala> Seq((1, Seq(Person("Jack", 5)))).toDF("id", "persons").withColumn("ages", udf{ s: Seq[Person] => s.head.age }.apply(col("persons"))).show

```

Before:

```

org.apache.spark.SparkException: Failed to execute user defined function($read$$Lambda$2861/628175152: (array<struct<name:string,age:int>>) => int)
  at org.apache.spark.sql.catalyst.expressions.ScalaUDF.eval(ScalaUDF.scala:1129)
  at org.apache.spark.sql.catalyst.expressions.Alias.eval(namedExpressions.scala:156)
  at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.apply(InterpretedMutableProjection.scala:83)
  at org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation$$anonfun$apply$17.$anonfun$applyOrElse$69(Optimizer.scala:1492)
  at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)

....

Caused by: java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema cannot be cast to Person
  at $anonfun$res3$1(<console>:30)
  at $anonfun$res3$1$adapted(<console>:30)
  at org.apache.spark.sql.catalyst.expressions.ScalaUDF.$anonfun$f$2(ScalaUDF.scala:156)
  at org.apache.spark.sql.catalyst.expressions.ScalaUDF.eval(ScalaUDF.scala:1126)
  ... 142 more
```

After:
```
+---+-----------+----+
| id|    persons|ages|
+---+-----------+----+
|  1|[[Jack, 5]]| [5]|
+---+-----------+----+
```

### How was this patch tested?

Added tests.

Closes #28645 from Ngone51/impr-udf.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-19 12:45:47 +00:00
Jungtaek Lim (HeartSaVioR) 6fe3bf66eb [SPARK-31993][SQL] Build arrays for passing variables generated from children for 'concat_ws' with columns having at least one of array type
### What changes were proposed in this pull request?

Please refer the next section `Why are the changes needed?` for details how the current implementation of `concat_ws` is broken for some condition.

This patch fixes the code generation logic for columns having at least one array types of columns in `concat_ws` to build two arrays for storing isNull and value from children's generated code and pass these arrays to the both varargCounts and varargBuilds. This change guarantees that both varargCounts and varargBuilds can access the relevant local variables the children's generated code makes as array parameters, which is critical to ensure both varargCounts and varargBuilds succeed to compile.

Below is the generated code for newly added UT, `SPARK-31993: concat_ws in agg function with plenty of string/array types columns`.

> before the patch

```
/* 001 */ public java.lang.Object generate(Object[] references) {
/* 002 */   return new SpecificUnsafeProjection(references);
/* 003 */ }
/* 004 */
/* 005 */ class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1];
/* 009 */
/* 010 */   public SpecificUnsafeProjection(Object[] references) {
/* 011 */     this.references = references;
/* 012 */     mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 32);
/* 013 */
/* 014 */   }
/* 015 */
/* 016 */   public void initialize(int partitionIndex) {
/* 017 */
/* 018 */   }
/* 019 */
/* 020 */   // Scala.Function1 need this
/* 021 */   public java.lang.Object apply(java.lang.Object row) {
/* 022 */     return apply((InternalRow) row);
/* 023 */   }
/* 024 */
/* 025 */   public UnsafeRow apply(InternalRow i) {
/* 026 */     mutableStateArray_0[0].reset();
/* 027 */
/* 028 */
/* 029 */     mutableStateArray_0[0].zeroOutNullBytes();
/* 030 */
/* 031 */     apply_0_0(i);
/* 032 */     apply_0_1(i);
/* 033 */     int varargNum_0 = 30;
/* 034 */     int idxInVararg_0 = 0;
/* 035 */
/* 036 */     if (!isNull_2) {
/* 037 */       varargNum_0 += value_2.numElements();
/* 038 */     }
/* 039 */
/* 040 */     if (!isNull_3) {
/* 041 */       varargNum_0 += value_3.numElements();
/* 042 */     }
/* 043 */
/* 044 */     UTF8String[] array_0 = new UTF8String[varargNum_0];
/* 045 */     idxInVararg_0 = varargBuildsConcatWs_0_0(i, array_0, idxInVararg_0);
/* 046 */     idxInVararg_0 = varargBuildsConcatWs_0_1(i, array_0, idxInVararg_0);
/* 047 */     idxInVararg_0 = varargBuildsConcatWs_0_2(i, array_0, idxInVararg_0);
/* 048 */     UTF8String value_0 = UTF8String.concatWs(((UTF8String) references[0] /* literal */), array_0);
/* 049 */     boolean isNull_0 = value_0 == null;
/* 050 */     mutableStateArray_0[0].write(0, value_0);
/* 051 */     return (mutableStateArray_0[0].getRow());
/* 052 */   }
/* 053 */
/* 054 */
/* 055 */   private void apply_0_1(InternalRow i) {
/* 056 */     UTF8String value_25 = i.getUTF8String(22);UTF8String value_26 = i.getUTF8String(23);UTF8String value_27 = i.getUTF8String(24);UTF8String value_28 = i.getUTF8String(25);UTF8String value_29 = i.getUTF8String(26);UTF8String value_30 = i.getUTF8String(27);UTF8String value_31 = i.getUTF8String(28);UTF8String value_32 = i.getUTF8String(29);UTF8String value_33 = i.getUTF8String(30);
/* 057 */   }
/* 058 */
/* 059 */
/* 060 */   private int varargBuildsConcatWs_0_0(InternalRow i, UTF8String [] array_0, int idxInVararg_0) {
/* 061 */
/* 062 */
/* 063 */     if (!isNull_2) {
/* 064 */       final int n_0 = value_2.numElements();
/* 065 */       for (int j = 0; j < n_0; j ++) {
/* 066 */         array_0[idxInVararg_0 ++] = value_2.getUTF8String(j);
/* 067 */       }
/* 068 */     }
/* 069 */
/* 070 */     if (!isNull_3) {
/* 071 */       final int n_1 = value_3.numElements();
/* 072 */       for (int j = 0; j < n_1; j ++) {
/* 073 */         array_0[idxInVararg_0 ++] = value_3.getUTF8String(j);
/* 074 */       }
/* 075 */     }
/* 076 */     array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_4;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_5;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_6;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_7;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_8;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_9;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_10;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_11;
/* 077 */     return idxInVararg_0;
/* 078 */
/* 079 */   }
/* 080 */
/* 081 */
/* 082 */   private int varargBuildsConcatWs_0_2(InternalRow i, UTF8String [] array_0, int idxInVararg_0) {
/* 083 */
/* 084 */     array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_28;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_29;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_30;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_31;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_32;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_33;
/* 085 */     return idxInVararg_0;
/* 086 */
/* 087 */   }
/* 088 */
/* 089 */
/* 090 */   private void apply_0_0(InternalRow i) {
/* 091 */     boolean isNull_2 = i.isNullAt(31);
/* 092 */     ArrayData value_2 = isNull_2 ?
/* 093 */     null : (i.getArray(31));boolean isNull_3 = i.isNullAt(32);
/* 094 */     ArrayData value_3 = isNull_3 ?
/* 095 */     null : (i.getArray(32));UTF8String value_4 = i.getUTF8String(1);UTF8String value_5 = i.getUTF8String(2);UTF8String value_6 = i.getUTF8String(3);UTF8String value_7 = i.getUTF8String(4);UTF8String value_8 = i.getUTF8String(5);UTF8String value_9 = i.getUTF8String(6);UTF8String value_10 = i.getUTF8String(7);UTF8String value_11 = i.getUTF8String(8);UTF8String value_12 = i.getUTF8String(9);UTF8String value_13 = i.getUTF8String(10);UTF8String value_14 = i.getUTF8String(11);UTF8String value_15 = i.getUTF8String(12);UTF8String value_16 = i.getUTF8String(13);UTF8String value_17 = i.getUTF8String(14);UTF8String value_18 = i.getUTF8String(15);UTF8String value_19 = i.getUTF8String(16);UTF8String value_20 = i.getUTF8String(17);UTF8String value_21 = i.getUTF8String(18);UTF8String value_22 = i.getUTF8String(19);UTF8String value_23 = i.getUTF8String(20);UTF8String value_24 = i.getUTF8String(21);
/* 096 */   }
/* 097 */
/* 098 */
/* 099 */   private int varargBuildsConcatWs_0_1(InternalRow i, UTF8String [] array_0, int idxInVararg_0) {
/* 100 */
/* 101 */     array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_12;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_13;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_14;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_15;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_16;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_17;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_18;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_19;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_20;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_21;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_22;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_23;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_24;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_25;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_26;array_0[idxInVararg_0 ++] = false ? (UTF8String) null : value_27;
/* 102 */     return idxInVararg_0;
/* 103 */
/* 104 */   }
/* 105 */
/* 106 */ }
```

Compilation of the generated code fails with error message: `org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 36, Column 6: Expression "isNull_2" is not an rvalue`

> after the patch

```
/* 001 */ public java.lang.Object generate(Object[] references) {
/* 002 */   return new SpecificUnsafeProjection(references);
/* 003 */ }
/* 004 */
/* 005 */ class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private boolean globalIsNull_0;
/* 009 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1];
/* 010 */
/* 011 */   public SpecificUnsafeProjection(Object[] references) {
/* 012 */     this.references = references;
/* 013 */
/* 014 */     mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 32);
/* 015 */
/* 016 */   }
/* 017 */
/* 018 */   public void initialize(int partitionIndex) {
/* 019 */
/* 020 */   }
/* 021 */
/* 022 */   // Scala.Function1 need this
/* 023 */   public java.lang.Object apply(java.lang.Object row) {
/* 024 */     return apply((InternalRow) row);
/* 025 */   }
/* 026 */
/* 027 */   public UnsafeRow apply(InternalRow i) {
/* 028 */     mutableStateArray_0[0].reset();
/* 029 */
/* 030 */
/* 031 */     mutableStateArray_0[0].zeroOutNullBytes();
/* 032 */
/* 033 */     UTF8String value_34 = ConcatWs_0(i);
/* 034 */     mutableStateArray_0[0].write(0, value_34);
/* 035 */     return (mutableStateArray_0[0].getRow());
/* 036 */   }
/* 037 */
/* 038 */
/* 039 */   private void initializeArgsArrays_0_0(InternalRow i, boolean [] isNullArgs_0, Object [] valueArgs_0) {
/* 040 */
/* 041 */     boolean isNull_2 = i.isNullAt(31);
/* 042 */     ArrayData value_2 = isNull_2 ?
/* 043 */     null : (i.getArray(31));
/* 044 */     isNullArgs_0[0] = isNull_2;
/* 045 */     valueArgs_0[0] = value_2;
/* 046 */
/* 047 */     boolean isNull_3 = i.isNullAt(32);
/* 048 */     ArrayData value_3 = isNull_3 ?
/* 049 */     null : (i.getArray(32));
/* 050 */     isNullArgs_0[1] = isNull_3;
/* 051 */     valueArgs_0[1] = value_3;
/* 052 */
/* 053 */     UTF8String value_4 = i.getUTF8String(1);
/* 054 */     isNullArgs_0[2] = false;
/* 055 */     valueArgs_0[2] = value_4;
/* 056 */
/* 057 */     UTF8String value_5 = i.getUTF8String(2);
/* 058 */     isNullArgs_0[3] = false;
/* 059 */     valueArgs_0[3] = value_5;
/* 060 */
/* 061 */     UTF8String value_6 = i.getUTF8String(3);
/* 062 */     isNullArgs_0[4] = false;
/* 063 */     valueArgs_0[4] = value_6;
/* 064 */
/* 065 */     UTF8String value_7 = i.getUTF8String(4);
/* 066 */     isNullArgs_0[5] = false;
/* 067 */     valueArgs_0[5] = value_7;
/* 068 */
/* 069 */     UTF8String value_8 = i.getUTF8String(5);
/* 070 */     isNullArgs_0[6] = false;
/* 071 */     valueArgs_0[6] = value_8;
/* 072 */
/* 073 */   }
/* 074 */
/* 075 */
/* 076 */   private void initializeArgsArrays_0_3(InternalRow i, boolean [] isNullArgs_0, Object [] valueArgs_0) {
/* 077 */
/* 078 */     UTF8String value_25 = i.getUTF8String(22);
/* 079 */     isNullArgs_0[23] = false;
/* 080 */     valueArgs_0[23] = value_25;
/* 081 */
/* 082 */     UTF8String value_26 = i.getUTF8String(23);
/* 083 */     isNullArgs_0[24] = false;
/* 084 */     valueArgs_0[24] = value_26;
/* 085 */
/* 086 */     UTF8String value_27 = i.getUTF8String(24);
/* 087 */     isNullArgs_0[25] = false;
/* 088 */     valueArgs_0[25] = value_27;
/* 089 */
/* 090 */     UTF8String value_28 = i.getUTF8String(25);
/* 091 */     isNullArgs_0[26] = false;
/* 092 */     valueArgs_0[26] = value_28;
/* 093 */
/* 094 */     UTF8String value_29 = i.getUTF8String(26);
/* 095 */     isNullArgs_0[27] = false;
/* 096 */     valueArgs_0[27] = value_29;
/* 097 */
/* 098 */     UTF8String value_30 = i.getUTF8String(27);
/* 099 */     isNullArgs_0[28] = false;
/* 100 */     valueArgs_0[28] = value_30;
/* 101 */
/* 102 */     UTF8String value_31 = i.getUTF8String(28);
/* 103 */     isNullArgs_0[29] = false;
/* 104 */     valueArgs_0[29] = value_31;
/* 105 */
/* 106 */     UTF8String value_32 = i.getUTF8String(29);
/* 107 */     isNullArgs_0[30] = false;
/* 108 */     valueArgs_0[30] = value_32;
/* 109 */
/* 110 */   }
/* 111 */
/* 112 */
/* 113 */   private int varargBuildsConcatWs_0_3(InternalRow i, UTF8String [] array_0, int idxInVararg_0, boolean [] isNullArgs_0, Object [] valueArgs_0) {
/* 114 */
/* 115 */     array_0[idxInVararg_0 ++] = isNullArgs_0[29] ? (UTF8String) null : ((UTF8String) valueArgs_0[29]);array_0[idxInVararg_0 ++] = isNullArgs_0[30] ? (UTF8String) null : ((UTF8String) valueArgs_0[30]);array_0[idxInVararg_0 ++] = isNullArgs_0[31] ? (UTF8String) null : ((UTF8String) valueArgs_0[31]);
/* 116 */     return idxInVararg_0;
/* 117 */
/* 118 */   }
/* 119 */
/* 120 */
/* 121 */   private int varargBuildsConcatWs_0_0(InternalRow i, UTF8String [] array_0, int idxInVararg_0, boolean [] isNullArgs_0, Object [] valueArgs_0) {
/* 122 */
/* 123 */
/* 124 */     if (!isNullArgs_0[0]) {
/* 125 */       final int n_0 = ((ArrayData) valueArgs_0[0]).numElements();
/* 126 */       for (int j = 0; j < n_0; j ++) {
/* 127 */         array_0[idxInVararg_0 ++] = ((ArrayData) valueArgs_0[0]).getUTF8String(j);
/* 128 */       }
/* 129 */     }
/* 130 */
/* 131 */     if (!isNullArgs_0[1]) {
/* 132 */       final int n_1 = ((ArrayData) valueArgs_0[1]).numElements();
/* 133 */       for (int j = 0; j < n_1; j ++) {
/* 134 */         array_0[idxInVararg_0 ++] = ((ArrayData) valueArgs_0[1]).getUTF8String(j);
/* 135 */       }
/* 136 */     }
/* 137 */     array_0[idxInVararg_0 ++] = isNullArgs_0[2] ? (UTF8String) null : ((UTF8String) valueArgs_0[2]);array_0[idxInVararg_0 ++] = isNullArgs_0[3] ? (UTF8String) null : ((UTF8String) valueArgs_0[3]);array_0[idxInVararg_0 ++] = isNullArgs_0[4] ? (UTF8String) null : ((UTF8String) valueArgs_0[4]);array_0[idxInVararg_0 ++] = isNullArgs_0[5] ? (UTF8String) null : ((UTF8String) valueArgs_0[5]);array_0[idxInVararg_0 ++] = isNullArgs_0[6] ? (UTF8String) null : ((UTF8String) valueArgs_0[6]);
/* 138 */     return idxInVararg_0;
/* 139 */
/* 140 */   }
/* 141 */
/* 142 */
/* 143 */   private UTF8String ConcatWs_0(InternalRow i) {
/* 144 */     boolean[] isNullArgs_0 = new boolean[32];
/* 145 */     Object[] valueArgs_0 = new Object[32];
/* 146 */     initializeArgsArrays_0_0(i, isNullArgs_0, valueArgs_0);
/* 147 */     initializeArgsArrays_0_1(i, isNullArgs_0, valueArgs_0);
/* 148 */     initializeArgsArrays_0_2(i, isNullArgs_0, valueArgs_0);
/* 149 */     initializeArgsArrays_0_3(i, isNullArgs_0, valueArgs_0);
/* 150 */     initializeArgsArrays_0_4(i, isNullArgs_0, valueArgs_0);
/* 151 */     int varargNum_0 = 30;
/* 152 */     int idxInVararg_0 = 0;
/* 153 */
/* 154 */     if (!isNullArgs_0[0]) {
/* 155 */       varargNum_0 += ((ArrayData) valueArgs_0[0]).numElements();
/* 156 */     }
/* 157 */
/* 158 */     if (!isNullArgs_0[1]) {
/* 159 */       varargNum_0 += ((ArrayData) valueArgs_0[1]).numElements();
/* 160 */     }
/* 161 */
/* 162 */     UTF8String[] array_0 = new UTF8String[varargNum_0];
/* 163 */     idxInVararg_0 = varargBuildsConcatWs_0_0(i, array_0, idxInVararg_0, isNullArgs_0, valueArgs_0);
/* 164 */     idxInVararg_0 = varargBuildsConcatWs_0_1(i, array_0, idxInVararg_0, isNullArgs_0, valueArgs_0);
/* 165 */     idxInVararg_0 = varargBuildsConcatWs_0_2(i, array_0, idxInVararg_0, isNullArgs_0, valueArgs_0);
/* 166 */     idxInVararg_0 = varargBuildsConcatWs_0_3(i, array_0, idxInVararg_0, isNullArgs_0, valueArgs_0);
/* 167 */     UTF8String value_0 = UTF8String.concatWs(((UTF8String) references[0] /* literal */), array_0);
/* 168 */     boolean isNull_0 = value_0 == null;
/* 169 */     globalIsNull_0 = isNull_0;
/* 170 */     return value_0;
/* 171 */   }
/* 172 */
/* 173 */
/* 174 */   private void initializeArgsArrays_0_2(InternalRow i, boolean [] isNullArgs_0, Object [] valueArgs_0) {
/* 175 */
/* 176 */     UTF8String value_17 = i.getUTF8String(14);
/* 177 */     isNullArgs_0[15] = false;
/* 178 */     valueArgs_0[15] = value_17;
/* 179 */
/* 180 */     UTF8String value_18 = i.getUTF8String(15);
/* 181 */     isNullArgs_0[16] = false;
/* 182 */     valueArgs_0[16] = value_18;
/* 183 */
/* 184 */     UTF8String value_19 = i.getUTF8String(16);
/* 185 */     isNullArgs_0[17] = false;
/* 186 */     valueArgs_0[17] = value_19;
/* 187 */
/* 188 */     UTF8String value_20 = i.getUTF8String(17);
/* 189 */     isNullArgs_0[18] = false;
/* 190 */     valueArgs_0[18] = value_20;
/* 191 */
/* 192 */     UTF8String value_21 = i.getUTF8String(18);
/* 193 */     isNullArgs_0[19] = false;
/* 194 */     valueArgs_0[19] = value_21;
/* 195 */
/* 196 */     UTF8String value_22 = i.getUTF8String(19);
/* 197 */     isNullArgs_0[20] = false;
/* 198 */     valueArgs_0[20] = value_22;
/* 199 */
/* 200 */     UTF8String value_23 = i.getUTF8String(20);
/* 201 */     isNullArgs_0[21] = false;
/* 202 */     valueArgs_0[21] = value_23;
/* 203 */
/* 204 */     UTF8String value_24 = i.getUTF8String(21);
/* 205 */     isNullArgs_0[22] = false;
/* 206 */     valueArgs_0[22] = value_24;
/* 207 */
/* 208 */   }
/* 209 */
/* 210 */
/* 211 */   private int varargBuildsConcatWs_0_2(InternalRow i, UTF8String [] array_0, int idxInVararg_0, boolean [] isNullArgs_0, Object [] valueArgs_0) {
/* 212 */
/* 213 */     array_0[idxInVararg_0 ++] = isNullArgs_0[18] ? (UTF8String) null : ((UTF8String) valueArgs_0[18]);array_0[idxInVararg_0 ++] = isNullArgs_0[19] ? (UTF8String) null : ((UTF8String) valueArgs_0[19]);array_0[idxInVararg_0 ++] = isNullArgs_0[20] ? (UTF8String) null : ((UTF8String) valueArgs_0[20]);array_0[idxInVararg_0 ++] = isNullArgs_0[21] ? (UTF8String) null : ((UTF8String) valueArgs_0[21]);array_0[idxInVararg_0 ++] = isNullArgs_0[22] ? (UTF8String) null : ((UTF8String) valueArgs_0[22]);array_0[idxInVararg_0 ++] = isNullArgs_0[23] ? (UTF8String) null : ((UTF8String) valueArgs_0[23]);array_0[idxInVararg_0 ++] = isNullArgs_0[24] ? (UTF8String) null : ((UTF8String) valueArgs_0[24]);array_0[idxInVararg_0 ++] = isNullArgs_0[25] ? (UTF8String) null : ((UTF8String) valueArgs_0[25]);array_0[idxInVararg_0 ++] = isNullArgs_0[26] ? (UTF8String) null : ((UTF8String) valueArgs_0[26]);array_0[idxInVararg_0 ++] = isNullArgs_0[27] ? (UTF8String) null : ((UTF8String) valueArgs_0[27]);array_0[idxInVararg_0 ++] = isNullArgs_0[28] ? (UTF8String) null : ((UTF8String) valueArgs_0[28]);
/* 214 */     return idxInVararg_0;
/* 215 */
/* 216 */   }
/* 217 */
/* 218 */
/* 219 */   private void initializeArgsArrays_0_4(InternalRow i, boolean [] isNullArgs_0, Object [] valueArgs_0) {
/* 220 */
/* 221 */     UTF8String value_33 = i.getUTF8String(30);
/* 222 */     isNullArgs_0[31] = false;
/* 223 */     valueArgs_0[31] = value_33;
/* 224 */
/* 225 */   }
/* 226 */
/* 227 */
/* 228 */   private void initializeArgsArrays_0_1(InternalRow i, boolean [] isNullArgs_0, Object [] valueArgs_0) {
/* 229 */
/* 230 */     UTF8String value_9 = i.getUTF8String(6);
/* 231 */     isNullArgs_0[7] = false;
/* 232 */     valueArgs_0[7] = value_9;
/* 233 */
/* 234 */     UTF8String value_10 = i.getUTF8String(7);
/* 235 */     isNullArgs_0[8] = false;
/* 236 */     valueArgs_0[8] = value_10;
/* 237 */
/* 238 */     UTF8String value_11 = i.getUTF8String(8);
/* 239 */     isNullArgs_0[9] = false;
/* 240 */     valueArgs_0[9] = value_11;
/* 241 */
/* 242 */     UTF8String value_12 = i.getUTF8String(9);
/* 243 */     isNullArgs_0[10] = false;
/* 244 */     valueArgs_0[10] = value_12;
/* 245 */
/* 246 */     UTF8String value_13 = i.getUTF8String(10);
/* 247 */     isNullArgs_0[11] = false;
/* 248 */     valueArgs_0[11] = value_13;
/* 249 */
/* 250 */     UTF8String value_14 = i.getUTF8String(11);
/* 251 */     isNullArgs_0[12] = false;
/* 252 */     valueArgs_0[12] = value_14;
/* 253 */
/* 254 */     UTF8String value_15 = i.getUTF8String(12);
/* 255 */     isNullArgs_0[13] = false;
/* 256 */     valueArgs_0[13] = value_15;
/* 257 */
/* 258 */     UTF8String value_16 = i.getUTF8String(13);
/* 259 */     isNullArgs_0[14] = false;
/* 260 */     valueArgs_0[14] = value_16;
/* 261 */
/* 262 */   }
/* 263 */
/* 264 */
/* 265 */   private int varargBuildsConcatWs_0_1(InternalRow i, UTF8String [] array_0, int idxInVararg_0, boolean [] isNullArgs_0, Object [] valueArgs_0) {
/* 266 */
/* 267 */     array_0[idxInVararg_0 ++] = isNullArgs_0[7] ? (UTF8String) null : ((UTF8String) valueArgs_0[7]);array_0[idxInVararg_0 ++] = isNullArgs_0[8] ? (UTF8String) null : ((UTF8String) valueArgs_0[8]);array_0[idxInVararg_0 ++] = isNullArgs_0[9] ? (UTF8String) null : ((UTF8String) valueArgs_0[9]);array_0[idxInVararg_0 ++] = isNullArgs_0[10] ? (UTF8String) null : ((UTF8String) valueArgs_0[10]);array_0[idxInVararg_0 ++] = isNullArgs_0[11] ? (UTF8String) null : ((UTF8String) valueArgs_0[11]);array_0[idxInVararg_0 ++] = isNullArgs_0[12] ? (UTF8String) null : ((UTF8String) valueArgs_0[12]);array_0[idxInVararg_0 ++] = isNullArgs_0[13] ? (UTF8String) null : ((UTF8String) valueArgs_0[13]);array_0[idxInVararg_0 ++] = isNullArgs_0[14] ? (UTF8String) null : ((UTF8String) valueArgs_0[14]);array_0[idxInVararg_0 ++] = isNullArgs_0[15] ? (UTF8String) null : ((UTF8String) valueArgs_0[15]);array_0[idxInVararg_0 ++] = isNullArgs_0[16] ? (UTF8String) null : ((UTF8String) valueArgs_0[16]);array_0[idxInVararg_0 ++] = isNullArgs_0[17] ? (UTF8String) null : ((UTF8String) valueArgs_0[17]);
/* 268 */     return idxInVararg_0;
/* 269 */
/* 270 */   }
/* 271 */
/* 272 */ }
```

### Why are the changes needed?

The generated code in `concat_ws` fails to compile when the below conditions are met:

* Plenty of columns are provided as input of `concat_ws`.
* There's at least one column with array[string] type. (In other words, not all columns are string type.)
* Splitting methods is triggered in `splitExpressionsWithCurrentInputs`.
  * This is a bit tricky, as the method won't split methods under whole stage codegen, as well as it will be simply no-op (inlined) if the number of blocks to convert into methods is 1.

a0187cd6b5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala (L88-L195)

There're three parts of generated code in `concat_ws` (`codes`, `varargCounts`, `varargBuilds`) and all parts try to split method by itself, while `varargCounts` and `varargBuilds` refer on the generated code in `codes`, hence the overall generated code fails to compile if any of part succeeds to split.

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

No.

### How was this patch tested?

New UTs added. (One for verification of the patch, another one for regression test)

Closes #28831 from HeartSaVioR/SPARK-31993.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-19 06:01:06 +00:00
Yuanjian Li 86b54f3321 [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store
### What changes were proposed in this pull request?
Introduce UnsafeRow format validation for streaming state store.

### Why are the changes needed?
Currently, Structured Streaming directly puts the UnsafeRow into StateStore without any schema validation. It's a dangerous behavior when users reusing the checkpoint file during migration. Any changes or bug fix related to the aggregate function may cause random exceptions, even the wrong answer, e.g SPARK-28067.

### Does this PR introduce _any_ user-facing change?
Yes. If the underlying changes are detected when the checkpoint is reused during migration, the InvalidUnsafeRowException will be thrown.

### How was this patch tested?
UT added. Will also add integrated tests for more scenario in another PR separately.

Closes #28707 from xuanyuanking/SPARK-31894.

Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-19 05:56:50 +00:00
Max Gekk 17a5007fd8 [SPARK-30865][SQL][SS] Refactor DateTimeUtils
### What changes were proposed in this pull request?

1. Move TimeZoneUTC and TimeZoneGMT to DateTimeTestUtils
2. Remove TimeZoneGMT
3. Use ZoneId.systemDefault() instead of defaultTimeZone().toZoneId
4. Alias SQLDate & SQLTimestamp to internal types of DateType and TimestampType
5. Avoid one `*` `DateTimeUtils`.`in fromJulianDay()`
6. Use toTotalMonths in `DateTimeUtils`.`subtractDates()`
7. Remove `julianCommonEraStart`, `timestampToString()`, `microsToEpochDays()`, `epochDaysToMicros()`, `instantToDays()` from `DateTimeUtils`.
8. Make splitDate() private.
9. Remove `def daysToMicros(days: Int): Long` and `def microsToDays(micros: Long): Int`.

### Why are the changes needed?

This simplifies the common code related to date-time operations, and should improve maintainability. In particular:

1. TimeZoneUTC and TimeZoneGMT are moved to DateTimeTestUtils because they are used only in tests
2. TimeZoneGMT can be removed because it is equal to TimeZoneUTC
3. After the PR #27494, Spark expressions and DateTimeUtils functions switched to ZoneId instead of TimeZone completely. `defaultTimeZone()` with `TimeZone` as return type is not needed anymore.
4. SQLDate and SQLTimestamp types can be explicitly aliased to internal types of DateType and and TimestampType instead of declaring this in a comment.
5. Avoid one `*` `DateTimeUtils`.`in fromJulianDay()`.
6. Use toTotalMonths in `DateTimeUtils`.`subtractDates()`.

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

### How was this patch tested?
By existing test suites

Closes #27617 from MaxGekk/move-time-zone-consts.

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-06-19 05:41:09 +00:00
Dilip Biswal e4f5036146 [SPARK-32020][SQL] Better error message when SPARK_HOME or spark.test.home is not set
### What changes were proposed in this pull request?
Better error message when SPARK_HOME or spark,test.home is not set.

### Why are the changes needed?
Currently the error message is not easily consumable as it prints  (see below) the real error after printing the current environment which is rather long.

**Old output**
`
 time.name" -> "Java(TM) SE Runtime Environment", "sun.boot.library.path" -> "/Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/lib",
 "java.vm.version" -> "25.221-b11",
 . . .
 . . .
 . . .
) did not contain key "SPARK_HOME" spark.test.home or SPARK_HOME is not set.
	at org.scalatest.Assertions.newAssertionFailedExceptio
`

**New output**
An exception or error caused a run to abort: spark.test.home or SPARK_HOME is not set.
org.scalatest.exceptions.TestFailedException: spark.test.home or SPARK_HOME is not set
### Does this PR introduce any user-facing change?
`
No.

### How was this patch tested?
Ran the tests in intellej  manually to see the new error.

Closes #28825 from dilipbiswal/minor-spark-31950-followup.

Authored-by: Dilip Biswal <dkbiswal@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-18 22:45:55 +09:00
Max Gekk 350aa859fe [SPARK-32006][SQL] Create date/timestamp formatters once before collect in hiveResultString()
### What changes were proposed in this pull request?
1. Add method `getTimeFormatters` to `HiveResult` which creates timestamp and date formatters.
2. Move creation of `dateFormatter` and `timestampFormatter` from the constructor of the `HiveResult` object to `HiveResult. hiveResultString()` via `getTimeFormatters`. This allows to resolve time zone ID from Spark's session time zone `spark.sql.session.timeZone` and create date/timestamp formatters only once before collecting `java.sql.Timestamp`/`java.sql.Date` values.
3. Create date/timestamp formatters once in SparkExecuteStatementOperation.

### Why are the changes needed?
To fix perf regression comparing to Spark 2.4

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

### How was this patch tested?
- By existing test suite `HiveResultSuite` and etc.
- Re-generate benchmarks results of `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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

Closes #28842 from MaxGekk/opt-toHiveString-oss-master.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-17 06:28:47 +00:00
Gabor Somogyi eeb81200e2 [SPARK-31337][SQL] Support MS SQL 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 MS SQL support.

What this PR contains:
* Added `MSSQLConnectionProvider`
* Added `MSSQLConnectionProviderSuite`
* Changed MS SQL JDBC driver to use the latest (test scope only)
* Changed `MsSqlServerIntegrationSuite` docker image to use the latest
* Added a version comment to `MariaDBConnectionProvider` to increase trackability

### 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 MS SQL using kerberos.

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

Closes #28635 from gaborgsomogyi/SPARK-31337.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@apache.org>
2020-06-16 18:22:12 -07:00
Max Gekk 36435658b1 [SPARK-31710][SQL][FOLLOWUP] Replace CAST by TIMESTAMP_SECONDS in benchmarks
### What changes were proposed in this pull request?
Replace `CAST(... AS TIMESTAMP` by `TIMESTAMP_SECONDS` in the following benchmarks:
- ExtractBenchmark
- DateTimeBenchmark
- FilterPushdownBenchmark
- InExpressionBenchmark

### Why are the changes needed?
The benchmarks fail w/o the changes:
```
[info] Running benchmark: datetime +/- interval
[info]   Running case: date + interval(m)
[error] Exception in thread "main" org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(`id` AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommend using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 5;
[error] 'Project [(cast(cast(id#0L as timestamp) as date) + 1 months) AS (CAST(CAST(id AS TIMESTAMP) AS DATE) + INTERVAL '1 months')#2]
[error] +- Range (0, 10000000, step=1, splits=Some(1))
```

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

### How was this patch tested?
By running the affected benchmarks.

Closes #28843 from MaxGekk/GuoPhilipse-31710-fix-compatibility-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-16 14:07:03 +00:00
GuoPhilipse f0e6d0ec13 [SPARK-31710][SQL] Fail casting numeric to timestamp by default
## What changes were proposed in this pull request?
we fail casting from numeric to timestamp by default.

## Why are the changes needed?
casting from numeric to timestamp is not a  non-standard,meanwhile it may generate different result between spark and other systems,for example hive

## Does this PR introduce any user-facing change?
Yes,user cannot cast numeric to timestamp directly,user have to use the following function to achieve the same effect:TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS

## How was this patch tested?
unit test added

Closes #28593 from GuoPhilipse/31710-fix-compatibility.

Lead-authored-by: GuoPhilipse <guofei_ok@126.com>
Co-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-16 08:35:35 +00:00
Jungtaek Lim (HeartSaVioR) fe68e95a5a [SPARK-24634][SS][FOLLOWUP] Rename the variable from "numLateInputs" to "numRowsDroppedByWatermark"
### What changes were proposed in this pull request?

This PR renames the variable from "numLateInputs" to "numRowsDroppedByWatermark" so that it becomes self-explanation.

### Why are the changes needed?

This is originated from post-review, see https://github.com/apache/spark/pull/28607#discussion_r439853232

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

No, as SPARK-24634 is not introduced in any release yet.

### How was this patch tested?

Existing UTs.

Closes #28828 from HeartSaVioR/SPARK-24634-v3-followup.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-16 16:41:08 +09:00
Takeshi Yamamuro 7f7b4dd519 [SPARK-31990][SS] Use toSet.toSeq in Dataset.dropDuplicates
### What changes were proposed in this pull request?

This PR partially revert SPARK-31292 in order to provide a hot-fix for a bug in `Dataset.dropDuplicates`; we must preserve the input order of `colNames` for `groupCols` because the Streaming's state store depends on the `groupCols` order.

### Why are the changes needed?

Bug fix.

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

No.

### How was this patch tested?

Added tests in `DataFrameSuite`.

Closes #28830 from maropu/SPARK-31990.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-15 07:48:48 -07:00
Max Gekk 9d95f1b010 [SPARK-31992][SQL] Benchmark the EXCEPTION rebase mode
### What changes were proposed in this pull request?
- Modify `DateTimeRebaseBenchmark` to benchmark the default date-time rebasing mode - `EXCEPTION` for saving/loading dates/timestamps from/to parquet files. The mode is benchmarked for modern timestamps after 1900-01-01 00:00:00Z and dates after 1582-10-15.
- Regenerate benchmark results 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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

### Why are the changes needed?
The `EXCEPTION` rebasing mode is the default mode of the SQL configs `spark.sql.legacy.parquet.datetimeRebaseModeInRead` and `spark.sql.legacy.parquet.datetimeRebaseModeInWrite`. The changes are needed to improve benchmark coverage for default settings.

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

### How was this patch tested?
By running the benchmark and check results manually.

Closes #28829 from MaxGekk/benchmark-exception-mode.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-15 07:25:56 +00:00
iRakson f5f6eee304 [SPARK-31642][FOLLOWUP] Fix Sorting for duration column and make Status column sortable
### What changes were proposed in this pull request?
In #28485 pagination support for tables of Structured Streaming Tab was added.
It missed 2 things:
* For sorting duration column, `String` was used which sometimes gives wrong results(consider `"3 ms"` and `"12 ms"`). Now we first sort the duration column and then convert it to readable String
* Status column was not made sortable.

### Why are the changes needed?
To fix the wrong result for sorting and making Status column sortable.

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

### How was this patch tested?
After changes:
<img width="1677" alt="Screenshot 2020-06-08 at 2 18 48 PM" src="https://user-images.githubusercontent.com/15366835/84010992-153fa280-a993-11ea-9846-bf176f2ec5d7.png">

Closes #28752 from iRakson/ssTests.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-14 16:41:59 -05:00
uncleGen 1e40bccf44 [SPARK-31593][SS] Remove unnecessary streaming query progress update
### What changes were proposed in this pull request?

Structured Streaming progress reporter will always report an `empty` progress when there is no new data. As design, we should provide progress updates every 10s (default) when there is no new data.

Before PR:

![20200428175008](https://user-images.githubusercontent.com/7402327/80474832-88a8ca00-897a-11ea-820b-d4be6127d2fe.jpg)
![20200428175037](https://user-images.githubusercontent.com/7402327/80474844-8ba3ba80-897a-11ea-873c-b7137bd4a804.jpg)
![20200428175102](https://user-images.githubusercontent.com/7402327/80474848-8e061480-897a-11ea-806e-28c6bbf1fe03.jpg)

After PR:

![image](https://user-images.githubusercontent.com/7402327/80475099-f35a0580-897a-11ea-8fb3-53f343df2c3f.png)

### Why are the changes needed?

Fixes a bug around incorrect progress report

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

Fixes a bug around incorrect progress report

### How was this patch tested?

existing ut and manual test

Closes #28391 from uncleGen/SPARK-31593.

Authored-by: uncleGen <hustyugm@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-14 14:49:01 +09:00
Jungtaek Lim (HeartSaVioR) 84815d0550 [SPARK-24634][SS] Add a new metric regarding number of inputs later than watermark plus allowed delay
### What changes were proposed in this pull request?

Please refer https://issues.apache.org/jira/browse/SPARK-24634 to see rationalization of the issue.

This patch adds a new metric to count the number of inputs arrived later than watermark plus allowed delay. To make changes simpler, this patch doesn't count the exact number of input rows which are later than watermark plus allowed delay. Instead, this patch counts the inputs which are dropped in the logic of operator. The difference of twos are shown in streaming aggregation: to optimize the calculation, streaming aggregation "pre-aggregates" the input rows, and later checks the lateness against "pre-aggregated" inputs, hence the number might be reduced.

The new metric will be provided via two places:

1. On Spark UI: check the metrics in stateful operator nodes in query execution details page in SQL tab
2. On Streaming Query Listener: check "numLateInputs" in "stateOperators" in QueryProcessEvent.

### Why are the changes needed?

Dropping late inputs means that end users might not get expected outputs. Even end users may indicate the fact and tolerate the result (as that's what allowed lateness is for), but they should be able to observe whether the current value of allowed lateness drops inputs or not so that they can adjust the value.

Also, whatever the chance they have multiple of stateful operators in a single query, if Spark drops late inputs "between" these operators, it becomes "correctness" issue. Spark should disallow such possibility, but given we already provided the flexibility, at least we should provide the way to observe the correctness issue and decide whether they should make correction of their query or not.

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

Yes. End users will be able to retrieve the information of late inputs via two ways:

1. SQL tab in Spark UI
2. Streaming Query Listener

### How was this patch tested?

New UTs added & existing UTs are modified to reflect the change.

And ran manual test reproducing SPARK-28094.

I've picked the specific case on "B outer C outer D" which is enough to represent the "intermediate late row" issue due to global watermark.

https://gist.github.com/jammann/b58bfbe0f4374b89ecea63c1e32c8f17

Spark logs warning message on the query which means SPARK-28074 is working correctly,

```
20/05/30 17:52:47 WARN UnsupportedOperationChecker: Detected pattern of possible 'correctness' issue due to global watermark. The query contains stateful operation which can emit rows older than the current watermark plus allowed late record delay, which are "late rows" in downstream stateful operations and these rows can be discarded. Please refer the programming guide doc for more details.;
Join LeftOuter, ((D_FK#28 = D_ID#87) AND (B_LAST_MOD#26-T30000ms = D_LAST_MOD#88-T30000ms))
:- Join LeftOuter, ((C_FK#27 = C_ID#58) AND (B_LAST_MOD#26-T30000ms = C_LAST_MOD#59-T30000ms))
:  :- EventTimeWatermark B_LAST_MOD#26: timestamp, 30 seconds
:  :  +- Project [v#23.B_ID AS B_ID#25, v#23.B_LAST_MOD AS B_LAST_MOD#26, v#23.C_FK AS C_FK#27, v#23.D_FK AS D_FK#28]
:  :     +- Project [from_json(StructField(B_ID,StringType,false), StructField(B_LAST_MOD,TimestampType,false), StructField(C_FK,StringType,true), StructField(D_FK,StringType,true), value#21, Some(UTC)) AS v#23]
:  :        +- Project [cast(value#8 as string) AS value#21]
:  :           +- StreamingRelationV2 org.apache.spark.sql.kafka010.KafkaSourceProvider3a7fd18c, kafka, org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaTable396d2958, org.apache.spark.sql.util.CaseInsensitiveStringMapa51ee61a, [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], StreamingRelation DataSource(org.apache.spark.sql.SparkSessiond221af8,kafka,List(),None,List(),None,Map(inferSchema -> true, startingOffsets -> earliest, subscribe -> B, kafka.bootstrap.servers -> localhost:9092),None), kafka, [key#0, value#1, topic#2, partition#3, offset#4L, timestamp#5, timestampType#6]
:  +- EventTimeWatermark C_LAST_MOD#59: timestamp, 30 seconds
:     +- Project [v#56.C_ID AS C_ID#58, v#56.C_LAST_MOD AS C_LAST_MOD#59]
:        +- Project [from_json(StructField(C_ID,StringType,false), StructField(C_LAST_MOD,TimestampType,false), value#54, Some(UTC)) AS v#56]
:           +- Project [cast(value#41 as string) AS value#54]
:              +- StreamingRelationV2 org.apache.spark.sql.kafka010.KafkaSourceProvider3f507373, kafka, org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaTable7b6736a4, org.apache.spark.sql.util.CaseInsensitiveStringMapa51ee61b, [key#40, value#41, topic#42, partition#43, offset#44L, timestamp#45, timestampType#46], StreamingRelation DataSource(org.apache.spark.sql.SparkSessiond221af8,kafka,List(),None,List(),None,Map(inferSchema -> true, startingOffsets -> earliest, subscribe -> C, kafka.bootstrap.servers -> localhost:9092),None), kafka, [key#33, value#34, topic#35, partition#36, offset#37L, timestamp#38, timestampType#39]
+- EventTimeWatermark D_LAST_MOD#88: timestamp, 30 seconds
   +- Project [v#85.D_ID AS D_ID#87, v#85.D_LAST_MOD AS D_LAST_MOD#88]
      +- Project [from_json(StructField(D_ID,StringType,false), StructField(D_LAST_MOD,TimestampType,false), value#83, Some(UTC)) AS v#85]
         +- Project [cast(value#70 as string) AS value#83]
            +- StreamingRelationV2 org.apache.spark.sql.kafka010.KafkaSourceProvider2b90e779, kafka, org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaTable36f8cd29, org.apache.spark.sql.util.CaseInsensitiveStringMapa51ee620, [key#69, value#70, topic#71, partition#72, offset#73L, timestamp#74, timestampType#75], StreamingRelation DataSource(org.apache.spark.sql.SparkSessiond221af8,kafka,List(),None,List(),None,Map(inferSchema -> true, startingOffsets -> earliest, subscribe -> D, kafka.bootstrap.servers -> localhost:9092),None), kafka, [key#62, value#63, topic#64, partition#65, offset#66L, timestamp#67, timestampType#68]
```

and we can find the late inputs from the batch 4 as follows:

![Screen Shot 2020-05-30 at 18 02 53](https://user-images.githubusercontent.com/1317309/83324401-058fd200-a2a0-11ea-8bf6-89cf777e9326.png)

which represents intermediate inputs are being lost, ended up with correctness issue.

Closes #28607 from HeartSaVioR/SPARK-24634-v3.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-14 14:37:38 +09:00
TJX2014 a4ea599b1b [SPARK-31968][SQL] Duplicate partition columns check when writing data
### What changes were proposed in this pull request?
A unit test is added
Partition duplicate check added in `org.apache.spark.sql.execution.datasources.PartitioningUtils#validatePartitionColumn`

### Why are the changes needed?
When people write data with duplicate partition column, it will cause a `org.apache.spark.sql.AnalysisException: Found duplicate column ...` in loading data from the  writted.

### Does this PR introduce _any_ user-facing change?
Yes.
It will prevent people from using duplicate partition columns to write data.
1. Before the PR:
It will look ok at `df.write.partitionBy("b", "b").csv("file:///tmp/output")`,
but get an exception when read:
`spark.read.csv("file:///tmp/output").show()`
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the partition schema: `b`;
2. After the PR:
`df.write.partitionBy("b", "b").csv("file:///tmp/output")` will trigger the exception:
org.apache.spark.sql.AnalysisException: Found duplicate column(s) b, b: `b`;

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

Closes #28814 from TJX2014/master-SPARK-31968.

Authored-by: TJX2014 <xiaoxingstack@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-13 22:21:35 -07:00
Liang-Chi Hsieh ff89b11143 [SPARK-31736][SQL] Nested column aliasing for RepartitionByExpression/Join
### What changes were proposed in this pull request?

Currently we only push nested column pruning through a few operators such as LIMIT, SAMPLE, etc. This patch extends the feature to other operators including RepartitionByExpression, Join.

### Why are the changes needed?

Currently nested column pruning only applied on a few operators. It limits the benefit of nested column pruning. Extending nested column pruning coverage to make this feature more generally applied through different queries.

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

Yes. More SQL operators are covered by nested column pruning.

### How was this patch tested?

Added unit test, end-to-end tests.

Closes #28556 from viirya/others-column-pruning.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-12 16:54:55 +09:00
Kousuke Saruta 88a4e55fae [SPARK-31765][WEBUI][TEST-MAVEN] Upgrade HtmlUnit >= 2.37.0
### What changes were proposed in this pull request?

This PR upgrades HtmlUnit.
Selenium and Jetty also upgraded because of dependency.
### Why are the changes needed?

Recently, a security issue which affects HtmlUnit is reported.
https://nvd.nist.gov/vuln/detail/CVE-2020-5529
According to the report, arbitrary code can be run by malicious users.
HtmlUnit is used for test so the impact might not be large but it's better to upgrade it just in case.

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

No.

### How was this patch tested?

Existing testcases.

Closes #28585 from sarutak/upgrade-htmlunit.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-11 18:27:53 -05:00
Takeshi Yamamuro b1adc3deee [SPARK-21117][SQL] Built-in SQL Function Support - WIDTH_BUCKET
### What changes were proposed in this pull request?

This PR intends to add a build-in SQL function - `WIDTH_BUCKET`.
It is the rework of #18323.

Closes #18323

The other RDBMS references for `WIDTH_BUCKET`:
 - Oracle: https://docs.oracle.com/cd/B28359_01/olap.111/b28126/dml_functions_2137.htm#OLADM717
 - PostgreSQL: https://www.postgresql.org/docs/current/functions-math.html
 - Snowflake: https://docs.snowflake.com/en/sql-reference/functions/width_bucket.html
 - Prestodb: https://prestodb.io/docs/current/functions/math.html
 - Teradata: https://docs.teradata.com/reader/kmuOwjp1zEYg98JsB8fu_A/Wa8vw69cGzoRyNULHZeudg
 - DB2: https://www.ibm.com/support/producthub/db2/docs/content/SSEPGG_11.5.0/com.ibm.db2.luw.sql.ref.doc/doc/r0061483.html?pos=2

### 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 #28764 from maropu/SPARK-21117.

Lead-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Yuming Wang <wgyumg@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-11 14:15:28 -07:00
Wenchen Fan 6fb9c80da1 [SPARK-31958][SQL] normalize special floating numbers in subquery
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/23388 .

https://github.com/apache/spark/pull/23388 has an issue: it doesn't handle subquery expressions and assumes they will be turned into joins. However, this is not true for non-correlated subquery expressions.

This PR fixes this issue. It now doesn't skip `Subquery`, and subquery expressions will be handled by `OptimizeSubqueries`, which runs the optimizer with the subquery.

Note that, correlated subquery expressions will be handled twice: once in `OptimizeSubqueries`, once later when it becomes join. This is OK as `NormalizeFloatingNumbers` is idempotent now.

### Why are the changes needed?

fix a bug

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

yes, see the newly added test.

### How was this patch tested?

new test

Closes #28785 from cloud-fan/normalize.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-11 06:39:14 +00:00
Jungtaek Lim (HeartSaVioR) 4afe2b1bc9 [SPARK-28199][SS][FOLLOWUP] Remove package private in class/object in sql.execution package
### What changes were proposed in this pull request?

This PR proposes to remove package private in classes/objects in sql.execution package, as per SPARK-16964.

### Why are the changes needed?

This is per post-hoc review comment, see https://github.com/apache/spark/pull/24996#discussion_r437126445

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

No.

### How was this patch tested?

N/A

Closes #28790 from HeartSaVioR/SPARK-28199-FOLLOWUP-apply-SPARK-16964.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-10 21:32:16 -07:00
Gengliang Wang 76b5ed4ffa [SPARK-31935][SQL][TESTS][FOLLOWUP] Fix the test case for Hadoop2/3
### What changes were proposed in this pull request?

This PR updates the test case to accept Hadoop 2/3 error message correctly.

### Why are the changes needed?

SPARK-31935(#28760) breaks Hadoop 3.2 UT because Hadoop 2 and Hadoop 3 have different exception messages.
In https://github.com/apache/spark/pull/28791, there are two test suites missed the fix

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

No
### How was this patch tested?

Unit test

Closes #28796 from gengliangwang/SPARK-31926-followup.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-10 20:59:48 -07:00
manuzhang 5d7853750f [SPARK-31942] Revert "[SPARK-31864][SQL] Adjust AQE skew join trigger condition
### What changes were proposed in this pull request?
This reverts commit b9737c3c22 while keeping following changes

* set default value of `spark.sql.adaptive.skewJoin.skewedPartitionFactor` to 5
* improve tests
* remove unused imports

### Why are the changes needed?
As discussed in https://github.com/apache/spark/pull/28669#issuecomment-641044531, revert SPARK-31864 for optimizing skew join to work for extremely clustered keys.

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

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

Closes #28770 from manuzhang/spark-31942.

Authored-by: manuzhang <owenzhang1990@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-11 03:34:07 +00:00
Kent Yao 22dda6e18e [SPARK-31939][SQL][TEST-JAVA11] Fix Parsing day of year when year field pattern is missing
### What changes were proposed in this pull request?

If a datetime pattern contains no year field, the day of year field should not be ignored if exists

e.g.

```
spark-sql> select to_timestamp('31', 'DD');
1970-01-01 00:00:00
spark-sql> select to_timestamp('31 30', 'DD dd');
1970-01-30 00:00:00

spark.sql.legacy.timeParserPolicy legacy
spark-sql> select to_timestamp('31', 'DD');
1970-01-31 00:00:00
spark-sql> select to_timestamp('31 30', 'DD dd');
NULL
```

This PR only fixes some corner cases that use 'D' pattern to parse datetimes and there is w/o 'y'.

### Why are the changes needed?

fix some corner cases

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

yes, the day of year field will not be ignored

### How was this patch tested?

add unit tests.

Closes #28766 from yaooqinn/SPARK-31939.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-11 03:29:12 +00:00
Dongjoon Hyun c7d45c0e0b [SPARK-31935][SQL][TESTS][FOLLOWUP] Fix the test case for Hadoop2/3
### What changes were proposed in this pull request?

This PR updates the test case to accept Hadoop 2/3 error message correctly.

### Why are the changes needed?

SPARK-31935(https://github.com/apache/spark/pull/28760) breaks Hadoop 3.2 UT because Hadoop 2 and Hadoop 3 have different exception messages.

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

No.

### How was this patch tested?

Pass the Jenkins with both Hadoop 2/3 or do the following manually.

**Hadoop 2.7**
```
$ build/sbt "sql/testOnly *.FileBasedDataSourceSuite -- -z SPARK-31935"
...
[info] All tests passed.
```

**Hadoop 3.2**
```
$ build/sbt "sql/testOnly *.FileBasedDataSourceSuite -- -z SPARK-31935" -Phadoop-3.2
...
[info] All tests passed.
```

Closes #28791 from dongjoon-hyun/SPARK-31935.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-10 17:36:32 -07:00
HyukjinKwon 00d06cad56 [SPARK-31915][SQL][PYTHON] Resolve the grouping column properly per the case sensitivity in grouped and cogrouped pandas UDFs
### What changes were proposed in this pull request?

This is another approach to fix the issue. See the previous try https://github.com/apache/spark/pull/28745. It was too invasive so I took more conservative approach.

This PR proposes to resolve grouping attributes separately first so it can be properly referred when `FlatMapGroupsInPandas` and `FlatMapCoGroupsInPandas` are resolved without ambiguity.

Previously,

```python
from pyspark.sql.functions import *
df = spark.createDataFrame([[1, 1]], ["column", "Score"])
pandas_udf("column integer, Score float", PandasUDFType.GROUPED_MAP)
def my_pandas_udf(pdf):
    return pdf.assign(Score=0.5)

df.groupby('COLUMN').apply(my_pandas_udf).show()
```

was failed as below:

```
pyspark.sql.utils.AnalysisException: "Reference 'COLUMN' is ambiguous, could be: COLUMN, COLUMN.;"
```
because the unresolved `COLUMN` in `FlatMapGroupsInPandas` doesn't know which reference to take from the child projection.

After this fix, it resolves the child projection first with grouping keys and pass, to `FlatMapGroupsInPandas`, the attribute as a grouping key from the child projection that is positionally selected.

### Why are the changes needed?

To resolve grouping keys correctly.

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

Yes,

```python
from pyspark.sql.functions import *
df = spark.createDataFrame([[1, 1]], ["column", "Score"])
pandas_udf("column integer, Score float", PandasUDFType.GROUPED_MAP)
def my_pandas_udf(pdf):
    return pdf.assign(Score=0.5)

df.groupby('COLUMN').apply(my_pandas_udf).show()
```

```python
df1 = spark.createDataFrame([(1, 1)], ("column", "value"))
df2 = spark.createDataFrame([(1, 1)], ("column", "value"))

df1.groupby("COLUMN").cogroup(
    df2.groupby("COLUMN")
).applyInPandas(lambda r, l: r + l, df1.schema).show()
```

Before:

```
pyspark.sql.utils.AnalysisException: Reference 'COLUMN' is ambiguous, could be: COLUMN, COLUMN.;
```

```
pyspark.sql.utils.AnalysisException: cannot resolve '`COLUMN`' given input columns: [COLUMN, COLUMN, value, value];;
'FlatMapCoGroupsInPandas ['COLUMN], ['COLUMN], <lambda>(column#9L, value#10L, column#13L, value#14L), [column#22L, value#23L]
:- Project [COLUMN#9L, column#9L, value#10L]
:  +- LogicalRDD [column#9L, value#10L], false
+- Project [COLUMN#13L, column#13L, value#14L]
   +- LogicalRDD [column#13L, value#14L], false
```

After:

```
+------+-----+
|column|Score|
+------+-----+
|     1|  0.5|
+------+-----+
```

```
+------+-----+
|column|value|
+------+-----+
|     2|    2|
+------+-----+
```

### How was this patch tested?

Unittests were added and manually tested.

Closes #28777 from HyukjinKwon/SPARK-31915-another.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2020-06-10 15:54:07 -07:00
Wenchen Fan c400519322 [SPARK-31956][SQL] Do not fail if there is no ambiguous self join
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/28695 , to fix the problem completely.

The root cause is that, `df("col").as("name")` is not a column reference anymore, and should not have the special column metadata. However, this was broken in ba7adc4949 (diff-ac415c903887e49486ba542a65eec980L1050-L1053)

This PR fixes the regression, by strip the special column metadata in `Column.name`, which is the behavior before https://github.com/apache/spark/pull/28326 .

### Why are the changes needed?

Fix a regression. We shouldn't fail if there is no ambiguous self-join.

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

Yes, the query in the test can run now.

### How was this patch tested?

updated test

Closes #28783 from cloud-fan/self-join.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-10 13:11:24 -07:00
Liang-Chi Hsieh 43063e2db2 [SPARK-27217][SQL] Nested column aliasing for more operators which can prune nested column
### What changes were proposed in this pull request?

Currently we only push nested column pruning from a Project through a few operators such as LIMIT, SAMPLE, etc. There are a few operators like Aggregate, Expand which can prune nested columns by themselves, without a Project on top.

This patch extends the feature to those operators.

### Why are the changes needed?

Currently nested column pruning only applied on a few cases. It limits the benefit of nested column pruning. Extending nested column pruning coverage to make this feature more generally applied through different queries.

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

Yes. More SQL operators are covered by nested column pruning.

### How was this patch tested?

Added unit test, end-to-end tests.

Closes #28560 from viirya/SPARK-27217-2.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-10 18:08:47 +09:00
Takuya UESHIN 032d17933b [SPARK-31945][SQL][PYSPARK] Enable cache for the same Python function
### What changes were proposed in this pull request?

This PR proposes to make `PythonFunction` holds `Seq[Byte]` instead of `Array[Byte]` to be able to compare if the byte array has the same values for the cache manager.

### Why are the changes needed?

Currently the cache manager doesn't use the cache for `udf` if the `udf` is created again even if the functions is the same.

```py
>>> func = lambda x: x

>>> df = spark.range(1)
>>> df.select(udf(func)("id")).cache()
```
```py
>>> df.select(udf(func)("id")).explain()
== Physical Plan ==
*(2) Project [pythonUDF0#14 AS <lambda>(id)#12]
+- BatchEvalPython [<lambda>(id#0L)], [pythonUDF0#14]
 +- *(1) Range (0, 1, step=1, splits=12)
```

This is because `PythonFunction` holds `Array[Byte]`, and `equals` method of array equals only when the both array is the same instance.

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

Yes, if the user reuse the Python function for the UDF, the cache manager will detect the same function and use the cache for it.

### How was this patch tested?

I added a test case and manually.

```py
>>> df.select(udf(func)("id")).explain()
== Physical Plan ==
InMemoryTableScan [<lambda>(id)#12]
   +- InMemoryRelation [<lambda>(id)#12], StorageLevel(disk, memory, deserialized, 1 replicas)
         +- *(2) Project [pythonUDF0#5 AS <lambda>(id)#3]
            +- BatchEvalPython [<lambda>(id#0L)], [pythonUDF0#5]
               +- *(1) Range (0, 1, step=1, splits=12)
```

Closes #28774 from ueshin/issues/SPARK-31945/udf_cache.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-10 16:38:59 +09:00
Gengliang Wang f3771c6b47 [SPARK-31935][SQL] Hadoop file system config should be effective in data source options
### What changes were proposed in this pull request?

Mkae Hadoop file system config effective in data source options.

From `org.apache.hadoop.fs.FileSystem.java`:
```
  public static FileSystem get(URI uri, Configuration conf) throws IOException {
    String scheme = uri.getScheme();
    String authority = uri.getAuthority();

    if (scheme == null && authority == null) {     // use default FS
      return get(conf);
    }

    if (scheme != null && authority == null) {     // no authority
      URI defaultUri = getDefaultUri(conf);
      if (scheme.equals(defaultUri.getScheme())    // if scheme matches default
          && defaultUri.getAuthority() != null) {  // & default has authority
        return get(defaultUri, conf);              // return default
      }
    }

    String disableCacheName = String.format("fs.%s.impl.disable.cache", scheme);
    if (conf.getBoolean(disableCacheName, false)) {
      return createFileSystem(uri, conf);
    }

    return CACHE.get(uri, conf);
  }
```
Before changes, the file system configurations in data source options are not propagated in `DataSource.scala`.
After changes, we can specify authority and URI schema related configurations for scanning file systems.

This problem only exists in data source V1. In V2, we already use `sparkSession.sessionState.newHadoopConfWithOptions(options)` in `FileTable`.
### Why are the changes needed?

Allow users to specify authority and URI schema related Hadoop configurations for file source reading.

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

Yes, the file system related Hadoop configuration in data source option will be effective on reading.

### How was this patch tested?

Unit test

Closes #28760 from gengliangwang/ds_conf.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-06-09 12:15:07 -07:00
Kent Yao 6a424b93e5 [SPARK-31830][SQL] Consistent error handling for datetime formatting and parsing functions
### What changes were proposed in this pull request?
Currently, `date_format` and `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp`, `to_date`  have different exception handling behavior for formatting datetime values.

In this PR, we apply the exception handling behavior of `date_format` to `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date`.

In the phase of creating the datetime formatted or formating, exceptions will be raised.

e.g.

```java
spark-sql> select date_format(make_timestamp(1, 1 ,1,1,1,1), 'yyyyyyyyyyy-MM-aaa');
20/05/28 15:25:38 ERROR SparkSQLDriver: Failed in [select date_format(make_timestamp(1, 1 ,1,1,1,1), 'yyyyyyyyyyy-MM-aaa')]
org.apache.spark.SparkUpgradeException: You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'yyyyyyyyyyy-MM-aaa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
```

```java
spark-sql> select date_format(make_timestamp(1, 1 ,1,1,1,1), 'yyyyyyyyyyy-MM-AAA');
20/05/28 15:26:10 ERROR SparkSQLDriver: Failed in [select date_format(make_timestamp(1, 1 ,1,1,1,1), 'yyyyyyyyyyy-MM-AAA')]
java.lang.IllegalArgumentException: Illegal pattern character: A
```

```java
spark-sql> select date_format(make_timestamp(1,1,1,1,1,1), 'yyyyyyyyyyy-MM-dd');
20/05/28 15:23:23 ERROR SparkSQLDriver: Failed in [select date_format(make_timestamp(1,1,1,1,1,1), 'yyyyyyyyyyy-MM-dd')]
java.lang.ArrayIndexOutOfBoundsException: 11
	at java.time.format.DateTimeFormatterBuilder$NumberPrinterParser.format(DateTimeFormatterBuilder.java:2568)
```
In the phase of parsing, `DateTimeParseException | DateTimeException | ParseException` will be suppressed, but `SparkUpgradeException` will still be raised

e.g.

```java
spark-sql> set spark.sql.legacy.timeParserPolicy=exception;
spark.sql.legacy.timeParserPolicy	exception
spark-sql> select to_timestamp("2020-01-27T20:06:11.847-0800", "yyyy-MM-dd'T'HH:mm:ss.SSSz");
20/05/28 15:31:15 ERROR SparkSQLDriver: Failed in [select to_timestamp("2020-01-27T20:06:11.847-0800", "yyyy-MM-dd'T'HH:mm:ss.SSSz")]
org.apache.spark.SparkUpgradeException: You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2020-01-27T20:06:11.847-0800' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
```

```java
spark-sql> set spark.sql.legacy.timeParserPolicy=corrected;
spark.sql.legacy.timeParserPolicy	corrected
spark-sql> select to_timestamp("2020-01-27T20:06:11.847-0800", "yyyy-MM-dd'T'HH:mm:ss.SSSz");
NULL
spark-sql> set spark.sql.legacy.timeParserPolicy=legacy;
spark.sql.legacy.timeParserPolicy	legacy
spark-sql> select to_timestamp("2020-01-27T20:06:11.847-0800", "yyyy-MM-dd'T'HH:mm:ss.SSSz");
2020-01-28 12:06:11.847
```

### Why are the changes needed?
Consistency

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

Yes, invalid datetime patterns will fail `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` instead of resulting `NULL`

### How was this patch tested?

add more tests

Closes #28650 from yaooqinn/SPARK-31830.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-09 16:56:45 +00:00
Yuming Wang 1d1eacde9d [SPARK-31220][SQL] repartition obeys initialPartitionNum when adaptiveExecutionEnabled
### What changes were proposed in this pull request?
This PR makes `repartition`/`DISTRIBUTE BY` obeys [initialPartitionNum](af4248b2d6/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala (L446-L455)) when adaptive execution enabled.

### Why are the changes needed?
To make `DISTRIBUTE BY`/`GROUP BY` partitioned by same partition number.
How to reproduce:
```scala
spark.sql("CREATE TABLE spark_31220(id int)")
spark.sql("set spark.sql.adaptive.enabled=true")
spark.sql("set spark.sql.adaptive.coalescePartitions.initialPartitionNum=1000")
```

Before this PR:
```
scala> spark.sql("SELECT id from spark_31220 GROUP BY id").explain
== Physical Plan ==
AdaptiveSparkPlan(isFinalPlan=false)
+- HashAggregate(keys=[id#5], functions=[])
   +- Exchange hashpartitioning(id#5, 1000), true, [id=#171]
      +- HashAggregate(keys=[id#5], functions=[])
         +- FileScan parquet default.spark_31220[id#5]

scala> spark.sql("SELECT id from spark_31220 DISTRIBUTE BY id").explain
== Physical Plan ==
AdaptiveSparkPlan(isFinalPlan=false)
+- Exchange hashpartitioning(id#5, 200), false, [id=#179]
   +- FileScan parquet default.spark_31220[id#5]
```
After this PR:
```
scala> spark.sql("SELECT id from spark_31220 GROUP BY id").explain
== Physical Plan ==
AdaptiveSparkPlan(isFinalPlan=false)
+- HashAggregate(keys=[id#5], functions=[])
   +- Exchange hashpartitioning(id#5, 1000), true, [id=#171]
      +- HashAggregate(keys=[id#5], functions=[])
         +- FileScan parquet default.spark_31220[id#5]

scala> spark.sql("SELECT id from spark_31220 DISTRIBUTE BY id").explain
== Physical Plan ==
AdaptiveSparkPlan(isFinalPlan=false)
+- Exchange hashpartitioning(id#5, 1000), false, [id=#179]
   +- FileScan parquet default.spark_31220[id#5]
```

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

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

Closes #27986 from wangyum/SPARK-31220.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-09 16:07:22 +00:00
Max Gekk ddd8d5f5a0 [SPARK-31932][SQL][TESTS] Add date/timestamp benchmarks for HiveResult.hiveResultString()
### What changes were proposed in this pull request?
Add benchmarks for `HiveResult.hiveResultString()/toHiveString()` to measure throughput of `toHiveString` for the date/timestamp types:
- java.sql.Date/Timestamp
- java.time.Instant
- java.time.LocalDate

Benchmark results were generated 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 |

### Why are the changes needed?
To detect perf regressions of `toHiveString` in the future.

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

### How was this patch tested?
By running `DateTimeBenchmark` and check dataset content.

Closes #28757 from MaxGekk/benchmark-toHiveString.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-09 04:59:41 +00:00
Kent Yao 9d5b5d0a58 [SPARK-31879][SQL][TEST-JAVA11] Make week-based pattern invalid for formatting too
# What changes were proposed in this pull request?

After all these attempts https://github.com/apache/spark/pull/28692 and https://github.com/apache/spark/pull/28719 an https://github.com/apache/spark/pull/28727.
they all have limitations as mentioned in their discussions.

Maybe the only way is to forbid them all

### Why are the changes needed?

These week-based fields need Locale to express their semantics, the first day of the week varies from country to country.

From the Java doc of WeekFields
```java
    /**
     * Gets the first day-of-week.
     * <p>
     * The first day-of-week varies by culture.
     * For example, the US uses Sunday, while France and the ISO-8601 standard use Monday.
     * This method returns the first day using the standard {code DayOfWeek} enum.
     *
     * return the first day-of-week, not null
     */
    public DayOfWeek getFirstDayOfWeek() {
        return firstDayOfWeek;
    }
```

But for the SimpleDateFormat, the day-of-week is not localized

```
u	Day number of week (1 = Monday, ..., 7 = Sunday)	Number	1
```

Currently, the default locale we use is the US, so the result moved a day or a year or a week backward.

e.g.

For the date `2019-12-29(Sunday)`, in the Sunday Start system(e.g. en-US), it belongs to 2020 of week-based-year, in the Monday Start system(en-GB), it goes to 2019. the week-of-week-based-year(w) will be affected too

```sql
spark-sql> SELECT to_csv(named_struct('time', to_timestamp('2019-12-29', 'yyyy-MM-dd')), map('timestampFormat', 'YYYY', 'locale', 'en-US'));
2020
spark-sql> SELECT to_csv(named_struct('time', to_timestamp('2019-12-29', 'yyyy-MM-dd')), map('timestampFormat', 'YYYY', 'locale', 'en-GB'));
2019

spark-sql> SELECT to_csv(named_struct('time', to_timestamp('2019-12-29', 'yyyy-MM-dd')), map('timestampFormat', 'YYYY-ww-uu', 'locale', 'en-US'));
2020-01-01
spark-sql> SELECT to_csv(named_struct('time', to_timestamp('2019-12-29', 'yyyy-MM-dd')), map('timestampFormat', 'YYYY-ww-uu', 'locale', 'en-GB'));
2019-52-07

spark-sql> SELECT to_csv(named_struct('time', to_timestamp('2020-01-05', 'yyyy-MM-dd')), map('timestampFormat', 'YYYY-ww-uu', 'locale', 'en-US'));
2020-02-01
spark-sql> SELECT to_csv(named_struct('time', to_timestamp('2020-01-05', 'yyyy-MM-dd')), map('timestampFormat', 'YYYY-ww-uu', 'locale', 'en-GB'));
2020-01-07
```

For other countries, please refer to [First Day of the Week in Different Countries](http://chartsbin.com/view/41671)

### Does this PR introduce _any_ user-facing change?
With this change, user can not use 'YwuW',  but 'e' for 'u' instead. This can at least turn this not to be a silent data change.

### How was this patch tested?

add unit tests

Closes #28728 from yaooqinn/SPARK-31879-NEW2.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-05 08:14:01 +00:00
Takuya UESHIN 632b5bce23 [SPARK-31903][SQL][PYSPARK][R] Fix toPandas with Arrow enabled to show metrics in Query UI
### What changes were proposed in this pull request?

In `Dataset.collectAsArrowToR` and `Dataset.collectAsArrowToPython`, since the code block for `serveToStream` is run in the separate thread, `withAction` finishes as soon as it starts the thread. As a result, it doesn't collect the metrics of the actual action and Query UI shows the plan graph without metrics.

We should call `serveToStream` first, then `withAction` in it.

### Why are the changes needed?

When calling toPandas, usually Query UI shows each plan node's metric and corresponding Stage ID and Task ID:

```py
>>> df = spark.createDataFrame([(1, 10, 'abc'), (2, 20, 'def')], schema=['x', 'y', 'z'])
>>> df.toPandas()
   x   y    z
0  1  10  abc
1  2  20  def
```

![Screen Shot 2020-06-03 at 4 47 07 PM](https://user-images.githubusercontent.com/506656/83815735-bec22380-a675-11ea-8ecc-bf2954731f35.png)

but if Arrow execution is enabled, it shows only plan nodes and the duration is not correct:

```py
>>> spark.conf.set('spark.sql.execution.arrow.pyspark.enabled', True)
>>> df.toPandas()
   x   y    z
0  1  10  abc
1  2  20  def
```

![Screen Shot 2020-06-03 at 4 47 27 PM](https://user-images.githubusercontent.com/506656/83815804-de594c00-a675-11ea-933a-d0ffc0f534dd.png)

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

Yes, the Query UI will show the plan with the correct metrics.

### How was this patch tested?

I checked it manually in my local.

![Screen Shot 2020-06-04 at 3 19 41 PM](https://user-images.githubusercontent.com/506656/83816265-d77f0900-a676-11ea-84b8-2a8d80428bc6.png)

Closes #28730 from ueshin/issues/SPARK-31903/to_pandas_with_arrow_query_ui.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-05 12:53:58 +09:00
Wenchen Fan e61d0de11f Revert "[SPARK-31879][SQL] Using GB as default Locale for datetime formatters"
This reverts commit c59f51bcc2.
2020-06-04 01:54:22 +08:00
Wenchen Fan 349015dce0 fix compilation 2020-06-03 20:11:41 +08:00
Max Gekk 125a89ce08 [SPARK-31878][SQL] Create date formatter only once in HiveResult
### What changes were proposed in this pull request?
1. Replace `def dateFormatter` to `val dateFormatter`.
2. Modify the `date formatting in hive result` test in `HiveResultSuite` to check modified code on various time zones.

### Why are the changes needed?
To avoid creation of `DateFormatter` per every incoming date in `HiveResult.toHiveString`. This should eliminate unnecessary creation of `SimpleDateFormat` instances and compilation of the default pattern `yyyy-MM-dd`. The changes can speed up processing of legacy date values of the `java.sql.Date` type which is collected by default.

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

### How was this patch tested?
Modified a test in `HiveResultSuite`.

Closes #28687 from MaxGekk/HiveResult-val-dateFormatter.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-03 12:00:20 +00:00
Kent Yao afe95bd9ad [SPARK-31892][SQL] Disable week-based date filed for parsing
### What changes were proposed in this pull request?

This PR disables week-based date filed for parsing

closes #28674
### Why are the changes needed?

1. It's an un-fixable behavior change to fill the gap between SimpleDateFormat and DateTimeFormater and backward-compatibility for different JDKs.A lot of effort has been made to prove it at https://github.com/apache/spark/pull/28674

2. The existing behavior itself in 2.4 is confusing, e.g.

```sql
spark-sql> select to_timestamp('1', 'w');
1969-12-28 00:00:00
spark-sql> select to_timestamp('1', 'u');
1970-01-05 00:00:00
```
  The 'u' here seems not to go to the Monday of the first week in week-based form or the first day of the year in non-week-based form but go to the Monday of the second week in week-based form.

And, e.g.
```sql
spark-sql> select to_timestamp('2020 2020', 'YYYY yyyy');
2020-01-01 00:00:00
spark-sql> select to_timestamp('2020 2020', 'yyyy YYYY');
2019-12-29 00:00:00
spark-sql> select to_timestamp('2020 2020 1', 'YYYY yyyy w');
NULL
spark-sql> select to_timestamp('2020 2020 1', 'yyyy YYYY w');
2019-12-29 00:00:00
```

  I think we don't need to introduce all the weird behavior from Java

3. The current test coverage for week-based date fields is almost 0%, which indicates that we've never imagined using it.

4. Avoiding JDK bugs

https://issues.apache.org/jira/browse/SPARK-31880

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

Yes, the 'Y/W/w/u/F/E' pattern cannot be used datetime parsing functions.

### How was this patch tested?

more tests added

Closes #28706 from yaooqinn/SPARK-31892.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-03 06:49:58 +00:00
Kent Yao c59f51bcc2 [SPARK-31879][SQL] Using GB as default Locale for datetime formatters
# What changes were proposed in this pull request?

This PR switches the default Locale from the `US` to `GB` to change the behavior of the first day of the week from Sunday-started to Monday-started as same as v2.4

### Why are the changes needed?

#### cases
```sql
spark-sql> select to_timestamp('2020-1-1', 'YYYY-w-u');
2019-12-29 00:00:00
spark-sql> set spark.sql.legacy.timeParserPolicy=legacy;
spark.sql.legacy.timeParserPolicy	legacy
spark-sql> select to_timestamp('2020-1-1', 'YYYY-w-u');
2019-12-30 00:00:00
```

#### reasons

These week-based fields need Locale to express their semantics, the first day of the week varies from country to country.

From the Java doc of WeekFields
```java
    /**
     * Gets the first day-of-week.
     * <p>
     * The first day-of-week varies by culture.
     * For example, the US uses Sunday, while France and the ISO-8601 standard use Monday.
     * This method returns the first day using the standard {code DayOfWeek} enum.
     *
     * return the first day-of-week, not null
     */
    public DayOfWeek getFirstDayOfWeek() {
        return firstDayOfWeek;
    }
```

But for the SimpleDateFormat, the day-of-week is not localized

```
u	Day number of week (1 = Monday, ..., 7 = Sunday)	Number	1
```

Currently, the default locale we use is the US, so the result moved a day backward.

For other countries, please refer to [First Day of the Week in Different Countries](http://chartsbin.com/view/41671)

With this change, it restores the first day of week calculating for functions when using the default locale.

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

Yes, but the behavior change is used to restore the old one of v2.4

### How was this patch tested?

add unit tests

Closes #28692 from yaooqinn/SPARK-31879.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-03 06:07:53 +00:00
HyukjinKwon baafd4386c Revert "[SPARK-31765][WEBUI] Upgrade HtmlUnit >= 2.37.0"
This reverts commit e5c3463910.
2020-06-03 14:15:30 +09:00
Pablo Langa e4db3b5b17 [SPARK-29431][WEBUI] Improve Web UI / Sql tab visualization with cached dataframes
### What changes were proposed in this pull request?
With this pull request I want to improve the Web UI / SQL tab visualization. The principal problem that I find is when you have a cache in your plan, the SQL visualization don’t show any information about the part of the plan that has been cached.

Before the change
![image](https://user-images.githubusercontent.com/12819544/66587418-aa7f6280-eb8a-11e9-80cf-bf10d6c0abea.png)
After the change
![image](https://user-images.githubusercontent.com/12819544/66587526-ddc1f180-eb8a-11e9-92de-c3b3f5657b66.png)

### Why are the changes needed?
When we have a SQL plan with cached dataframes we lose the graphical information of this dataframe in the sql tab

### Does this PR introduce any user-facing change?
Yes, in the sql tab

### How was this patch tested?
Unit testing and manual tests throught spark shell

Closes #26082 from planga82/feature/SPARK-29431_SQL_Cache_webUI.

Lead-authored-by: Pablo Langa <soypab@gmail.com>
Co-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Unknown <soypab@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-06-02 17:26:43 -07:00
Kousuke Saruta e5c3463910 [SPARK-31765][WEBUI] Upgrade HtmlUnit >= 2.37.0
### What changes were proposed in this pull request?

This PR upgrades HtmlUnit.
Selenium and Jetty also upgraded because of dependency.
### Why are the changes needed?

Recently, a security issue which affects HtmlUnit is reported.
https://nvd.nist.gov/vuln/detail/CVE-2020-5529
According to the report, arbitrary code can be run by malicious users.
HtmlUnit is used for test so the impact might not be large but it's better to upgrade it just in case.

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

No.

### How was this patch tested?

Existing testcases.

Closes #28585 from sarutak/upgrade-htmlunit.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-02 08:29:07 -05:00
lipzhu d79a8a88b1 [SPARK-31834][SQL] Improve error message for incompatible data types
### What changes were proposed in this pull request?
We should use dataType.catalogString to unified the data type mismatch message.
Before:
```sql
spark-sql> create table SPARK_31834(a int) using parquet;
spark-sql> insert into SPARK_31834 select '1';
Error in query: Cannot write incompatible data to table '`default`.`spark_31834`':
- Cannot safely cast 'a': StringType to IntegerType;
```

After:
```sql
spark-sql> create table SPARK_31834(a int) using parquet;
spark-sql> insert into SPARK_31834 select '1';
Error in query: Cannot write incompatible data to table '`default`.`spark_31834`':
- Cannot safely cast 'a': string to int;
```

### How was this patch tested?
UT.

Closes #28654 from lipzhu/SPARK-31834.

Authored-by: lipzhu <lipzhu@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-02 21:07:10 +09:00
Max Gekk 00b355b97b [SPARK-31888][SQL] Support java.time.Instant in Parquet filter pushdown
### What changes were proposed in this pull request?
1. Modified `ParquetFilters.valueCanMakeFilterOn()` to accept filters with `java.time.Instant` attributes.
2. Added `ParquetFilters.timestampToMicros()` to support both types `java.sql.Timestamp` and `java.time.Instant` in conversions to microseconds.
3. Re-used `timestampToMicros` in constructing of Parquet filters.

### Why are the changes needed?
To support pushed down filters with `java.time.Instant` attributes. Before the changes, date filters are not pushed down to Parquet datasource when `spark.sql.datetime.java8API.enabled` is `true`.

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

### How was this patch tested?
Modified tests to `ParquetFilterSuite` to check the case when Java 8 API is enabled.

Closes #28696 from MaxGekk/support-instant-parquet-filters.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-02 11:53:58 +00:00
Sunitha Kambhampati 4161c62429 [SPARK-28067][SQL] Fix incorrect results for decimal aggregate sum by returning null on decimal overflow
### What changes were proposed in this pull request?

JIRA SPARK-28067:  Wrong results are returned for aggregate sum with decimals with whole stage codegen enabled

**Repro:**
WholeStage enabled enabled ->  Wrong results
WholeStage disabled -> Returns exception Decimal precision 39 exceeds max precision 38

**Issues:**
1. Wrong results are returned which is bad
2. Inconsistency between whole stage enabled and disabled.

**Cause:**
Sum does not take care of possibility of overflow for the intermediate steps.  ie the updateExpressions and mergeExpressions.

This PR makes the following changes:
- Add changes to check if overflow occurs for decimal in aggregate Sum and if there is an overflow,  it will return null for the Sum operation when spark.sql.ansi.enabled is false.
- When spark.sql.ansi.enabled is true, then the sum operation will return an exception if an overflow occurs for the decimal operation in Sum.
- This is keeping it consistent with the behavior defined in spark.sql.ansi.enabled property

**Before the fix:  Scenario 1:** - WRONG RESULTS
```
scala> val df = Seq(
     |  (BigDecimal("10000000000000000000"), 1),
     |  (BigDecimal("10000000000000000000"), 1),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2)).toDF("decNum", "intNum")
df: org.apache.spark.sql.DataFrame = [decNum: decimal(38,18), intNum: int]

scala> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, "intNum").agg(sum("decNum"))
df2: org.apache.spark.sql.DataFrame = [sum(decNum): decimal(38,18)]

scala> df2.show(40,false)
+---------------------------------------+
|sum(decNum)                            |
+---------------------------------------+
|20000000000000000000.000000000000000000|
+---------------------------------------+
```

--
**Before fix: Scenario2:  Setting spark.sql.ansi.enabled to true** - WRONG RESULTS
```
scala> spark.conf.set("spark.sql.ansi.enabled", "true")

scala> val df = Seq(
     |  (BigDecimal("10000000000000000000"), 1),
     |  (BigDecimal("10000000000000000000"), 1),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2)).toDF("decNum", "intNum")
df: org.apache.spark.sql.DataFrame = [decNum: decimal(38,18), intNum: int]

scala> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, "intNum").agg(sum("decNum"))
df2: org.apache.spark.sql.DataFrame = [sum(decNum): decimal(38,18)]

scala> df2.show(40,false)
+---------------------------------------+
|sum(decNum)                            |
+---------------------------------------+
|20000000000000000000.000000000000000000|
+---------------------------------------+

```

**After the fix: Scenario1:**
```
scala> val df = Seq(
     |  (BigDecimal("10000000000000000000"), 1),
     |  (BigDecimal("10000000000000000000"), 1),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2)).toDF("decNum", "intNum")
df: org.apache.spark.sql.DataFrame = [decNum: decimal(38,18), intNum: int]

scala> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, "intNum").agg(sum("decNum"))
df2: org.apache.spark.sql.DataFrame = [sum(decNum): decimal(38,18)]

scala>  df2.show(40,false)
+-----------+
|sum(decNum)|
+-----------+
|null       |
+-----------+

```

**After fix:  Scenario2:  Setting the spark.sql.ansi.enabled to true:**
```
scala> spark.conf.set("spark.sql.ansi.enabled", "true")

scala> val df = Seq(
     |  (BigDecimal("10000000000000000000"), 1),
     |  (BigDecimal("10000000000000000000"), 1),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2)).toDF("decNum", "intNum")
df: org.apache.spark.sql.DataFrame = [decNum: decimal(38,18), intNum: int]

scala> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, "intNum").agg(sum("decNum"))
df2: org.apache.spark.sql.DataFrame = [sum(decNum): decimal(38,18)]

scala>  df2.show(40,false)
20/02/18 10:57:43 ERROR Executor: Exception in task 5.0 in stage 4.0 (TID 30)
java.lang.ArithmeticException: Decimal(expanded,100000000000000000000.000000000000000000,39,18}) cannot be represented as Decimal(38, 18).

```

### Why are the changes needed?
The changes are needed in order to fix the wrong results that are returned for decimal aggregate sum.

### Does this PR introduce any user-facing change?
User would see wrong results on aggregate sum that involved decimal overflow prior to this change, but now the user will see null.  But if user enables the spark.sql.ansi.enabled flag to true, then the user will see an exception and not incorrect results.

### How was this patch tested?
New test has been added and existing tests for sql, catalyst and hive suites were run ok.

Closes #27627 from skambha/decaggfixwrongresults.

Lead-authored-by: Sunitha Kambhampati <skambha@us.ibm.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-02 11:30:30 +00:00
manuzhang 283814a426 [SPARK-31870][SQL][TESTS] Fix "Do not optimize skew join if additional shuffle" test having no skew join
### What changes were proposed in this pull request?
Fix configurations and ensure there is skew join in the test "Do not optimize skew join if additional shuffle".

### Why are the changes needed?
The existing "Do not optimize skew join if additional shuffle" test has no skew join at all.

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

### How was this patch tested?
Fixed existing test.

Closes #28679 from manuzhang/spark-31870.

Authored-by: manuzhang <owenzhang1990@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-02 02:00:58 +00:00
HyukjinKwon ea45fc5192 [SPARK-28344][SQL][FOLLOW-UP] Check the ambiguous self-join only if there is a join in the plan
### What changes were proposed in this pull request?

This PR proposes to check `DetectAmbiguousSelfJoin` only if there is `Join` in the plan. Currently, the checking is too strict even to non-join queries.

For example, the codes below don't have join at all but it fails as the ambiguous self-join:

```scala
import org.apache.spark.sql.expressions.Window
import org.apache.spark.sql.functions.sum
val df = Seq(1, 1, 2, 2).toDF("A")
val w = Window.partitionBy(df("A"))
df.select(df("A").alias("X"), sum(df("A")).over(w)).explain(true)
```

It is because `ExtractWindowExpressions` can create a `AttributeReference` with the same metadata but a different expression ID, see:

0fd98abd85/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala (L2679)
71c73d58f6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala (L63)
5945d46c11/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala (L180)

Before:

```
'Project [A#19 AS X#21, sum(A#19) windowspecdefinition(A#19, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS sum(A) OVER (PARTITION BY A unspecifiedframe$())#23L]
+- Relation[A#19] parquet
```

After:

```
Project [X#21, sum(A) OVER (PARTITION BY A unspecifiedframe$())#23L]
+- Project [X#21, A#19, sum(A) OVER (PARTITION BY A unspecifiedframe$())#23L, sum(A) OVER (PARTITION BY A unspecifiedframe$())#23L]
   +- Window [sum(A#19) windowspecdefinition(A#19, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS sum(A) OVER (PARTITION BY A unspecifiedframe$())#23L], [A#19]
      +- Project [A#19 AS X#21, A#19]
         +- Relation[A#19] parquet
```

`X#21` holds the same metadata of DataFrame ID and column position with `A#19` but it has a different expression ID which ends up with the checking fails.

### Why are the changes needed?

To loose the checking and make users not surprised.

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

It's the changes in unreleased branches only.

### How was this patch tested?

Manually tested and unittest was added.

Closes #28695 from HyukjinKwon/SPARK-28344-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-01 16:31:39 -07:00
Max Gekk 9c0dc28a6c [SPARK-31885][SQL] Fix filter push down for old millis timestamps to Parquet
### What changes were proposed in this pull request?
Fixed conversions of `java.sql.Timestamp` to milliseconds in `ParquetFilter` by using existing functions from `DateTimeUtils` `fromJavaTimestamp()` and `microsToMillis()`.

### Why are the changes needed?
The changes fix the bug:
```scala
scala> spark.conf.set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MILLIS")
scala> spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInWrite", "CORRECTED")
scala> Seq(java.sql.Timestamp.valueOf("1000-06-14 08:28:53.123")).toDF("ts").write.mode("overwrite").parquet("/Users/maximgekk/tmp/ts_millis_old_filter")
scala> spark.read.parquet("/Users/maximgekk/tmp/ts_millis_old_filter").filter($"ts" === "1000-06-14 08:28:53.123").show(false)
+---+
|ts |
+---+
+---+
```

### Does this PR introduce _any_ user-facing change?
Yes, after the changes (for the example above):
```scala
scala> spark.read.parquet("/Users/maximgekk/tmp/ts_millis_old_filter").filter($"ts" === "1000-06-14 08:28:53.123").show(false)
+-----------------------+
|ts                     |
+-----------------------+
|1000-06-14 08:28:53.123|
+-----------------------+
```

### How was this patch tested?
Modified tests in `ParquetFilterSuite` to check old timestamps.

Closes #28693 from MaxGekk/parquet-ts-millis-filter.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-01 15:13:44 +00:00
Takeshi Yamamuro b806fc4582 [SPARK-31854][SQL] Invoke in MapElementsExec should not propagate null
### What changes were proposed in this pull request?

This PR intends to fix a bug of `Dataset.map` below when the whole-stage codegen enabled;
```
scala> val ds = Seq(1.asInstanceOf[Integer], null.asInstanceOf[Integer]).toDS()

scala> sql("SET spark.sql.codegen.wholeStage=true")

scala> ds.map(v=>(v,v)).explain
== Physical Plan ==
*(1) SerializeFromObject [assertnotnull(input[0, scala.Tuple2, true])._1.intValue AS _1#69, assertnotnull(input[0, scala.Tuple2, true])._2.intValue AS _2#70]
+- *(1) MapElements <function1>, obj#68: scala.Tuple2
   +- *(1) DeserializeToObject staticinvoke(class java.lang.Integer, ObjectType(class java.lang.Integer), valueOf, value#1, true, false), obj#67: java.lang.Integer
      +- LocalTableScan [value#1]

// `AssertNotNull` in `SerializeFromObject` will fail;
scala> ds.map(v => (v, v)).show()
java.lang.NullPointerException: Null value appeared in non-nullable fails:
top level Product input object
If the schema is inferred from a Scala tuple/case class, or a Java bean, please try to use scala.Option[_] or other nullable types (e.g. java.lang.Integer instead of int/scala.Int).

// When the whole-stage codegen disabled, the query works well;
scala> sql("SET spark.sql.codegen.wholeStage=false")
scala> ds.map(v=>(v,v)).show()
+----+----+
|  _1|  _2|
+----+----+
|   1|   1|
|null|null|
+----+----+
```
A root cause is that `Invoke` used in `MapElementsExec` propagates input null, and then [AssertNotNull](1b780f364b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala (L253-L255)) in `SerializeFromObject` fails because a top-level row becomes null. So, `MapElementsExec` should not return `null` but `(null, null)`.

NOTE: the generated code of the query above in the current master;
```
/* 033 */   private void mapelements_doConsume_0(java.lang.Integer mapelements_expr_0_0, boolean mapelements_exprIsNull_0_0) throws java.io.IOException {
/* 034 */     boolean mapelements_isNull_1 = true;
/* 035 */     scala.Tuple2 mapelements_value_1 = null;
/* 036 */     if (!false) {
/* 037 */       mapelements_resultIsNull_0 = false;
/* 038 */
/* 039 */       if (!mapelements_resultIsNull_0) {
/* 040 */         mapelements_resultIsNull_0 = mapelements_exprIsNull_0_0;
/* 041 */         mapelements_mutableStateArray_0[0] = mapelements_expr_0_0;
/* 042 */       }
/* 043 */
/* 044 */       mapelements_isNull_1 = mapelements_resultIsNull_0;
/* 045 */       if (!mapelements_isNull_1) {
/* 046 */         Object mapelements_funcResult_0 = null;
/* 047 */         mapelements_funcResult_0 = ((scala.Function1) references[1] /* literal */).apply(mapelements_mutableStateArray_0[0]);
/* 048 */
/* 049 */         if (mapelements_funcResult_0 != null) {
/* 050 */           mapelements_value_1 = (scala.Tuple2) mapelements_funcResult_0;
/* 051 */         } else {
/* 052 */           mapelements_isNull_1 = true;
/* 053 */         }
/* 054 */
/* 055 */       }
/* 056 */     }
/* 057 */
/* 058 */     serializefromobject_doConsume_0(mapelements_value_1, mapelements_isNull_1);
/* 059 */
/* 060 */   }
```

The generated code w/ this fix;
```
/* 032 */   private void mapelements_doConsume_0(java.lang.Integer mapelements_expr_0_0, boolean mapelements_exprIsNull_0_0) throws java.io.IOException {
/* 033 */     boolean mapelements_isNull_1 = true;
/* 034 */     scala.Tuple2 mapelements_value_1 = null;
/* 035 */     if (!false) {
/* 036 */       mapelements_mutableStateArray_0[0] = mapelements_expr_0_0;
/* 037 */
/* 038 */       mapelements_isNull_1 = false;
/* 039 */       if (!mapelements_isNull_1) {
/* 040 */         Object mapelements_funcResult_0 = null;
/* 041 */         mapelements_funcResult_0 = ((scala.Function1) references[1] /* literal */).apply(mapelements_mutableStateArray_0[0]);
/* 042 */
/* 043 */         if (mapelements_funcResult_0 != null) {
/* 044 */           mapelements_value_1 = (scala.Tuple2) mapelements_funcResult_0;
/* 045 */           mapelements_isNull_1 = false;
/* 046 */         } else {
/* 047 */           mapelements_isNull_1 = true;
/* 048 */         }
/* 049 */
/* 050 */       }
/* 051 */     }
/* 052 */
/* 053 */     serializefromobject_doConsume_0(mapelements_value_1, mapelements_isNull_1);
/* 054 */
/* 055 */   }
```

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

Added tests.

Closes #28681 from maropu/SPARK-31854.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-01 04:50:00 +00:00
Kent Yao 547c5bf552 [SPARK-31867][SQL] Disable year type datetime patterns which are longer than 10
### What changes were proposed in this pull request?

As mentioned in https://github.com/apache/spark/pull/28673 and suggested via cloud-fan at https://github.com/apache/spark/pull/28673#discussion_r432817075

In this PR, we disable datetime pattern in the form of `y..y` and `Y..Y` whose lengths are greater than 10 to avoid sort of JDK bug as described below

he new datetime formatter introduces silent data change like,

```sql
spark-sql> select from_unixtime(1, 'yyyyyyyyyyy-MM-dd');
NULL
spark-sql> set spark.sql.legacy.timeParserPolicy=legacy;
spark.sql.legacy.timeParserPolicy	legacy
spark-sql> select from_unixtime(1, 'yyyyyyyyyyy-MM-dd');
00000001970-01-01
spark-sql>
```

For patterns that support `SignStyle.EXCEEDS_PAD`, e.g. `y..y`(len >=4), when using the `NumberPrinterParser` to format it

```java
switch (signStyle) {
  case EXCEEDS_PAD:
    if (minWidth < 19 && value >= EXCEED_POINTS[minWidth]) {
      buf.append(decimalStyle.getPositiveSign());
    }
    break;

           ....
```
the `minWidth` == `len(y..y)`
the `EXCEED_POINTS` is

```java
/**
         * Array of 10 to the power of n.
         */
        static final long[] EXCEED_POINTS = new long[] {
            0L,
            10L,
            100L,
            1000L,
            10000L,
            100000L,
            1000000L,
            10000000L,
            100000000L,
            1000000000L,
            10000000000L,
        };
```

So when the `len(y..y)` is greater than 10, ` ArrayIndexOutOfBoundsException` will be raised.

 And at the caller side, for `from_unixtime`, the exception will be suppressed and silent data change occurs. for `date_format`, the `ArrayIndexOutOfBoundsException` will continue.

### Why are the changes needed?
fix silent data change

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

Yes, SparkUpgradeException will take place of `null` result when the pattern contains 10 or more continuous  'y' or 'Y'

### How was this patch tested?

new tests

Closes #28684 from yaooqinn/SPARK-31867-2.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-31 12:34:39 +00:00
Maryann Xue b9737c3c22 [SPARK-31864][SQL] Adjust AQE skew join trigger condition
### What changes were proposed in this pull request?

This PR makes a minor change in deciding whether a partition is skewed by comparing the partition size to the median size of coalesced partitions instead of median size of raw partitions before coalescing.

### Why are the changes needed?

This change is line with target size criteria of splitting skew join partitions and can also cope with situations of extra empty partitions caused by over-partitioning. This PR has also improved skew join tests in AQE tests.

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

No.

### How was this patch tested?

Updated UTs.

Closes #28669 from maryannxue/spark-31864.

Authored-by: Maryann Xue <maryann.xue@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-30 07:47:29 +00:00
Yuming Wang 91148f428b [SPARK-28481][SQL] More expressions should extend NullIntolerant
### What changes were proposed in this pull request?

1. Make more expressions extend `NullIntolerant`.
2. Add a checker(in `ExpressionInfoSuite`) to identify whether the expression is `NullIntolerant`.

### Why are the changes needed?

Avoid skew join if the join column has many null values and can improve query performance. For examples:
```sql
CREATE TABLE t1(c1 string, c2 string) USING parquet;
CREATE TABLE t2(c1 string, c2 string) USING parquet;
EXPLAIN SELECT t1.* FROM t1 JOIN t2 ON upper(t1.c1) = upper(t2.c1);
```

Before and after this PR:
```sql
== Physical Plan ==
*(2) Project [c1#5, c2#6]
+- *(2) BroadcastHashJoin [upper(c1#5)], [upper(c1#7)], Inner, BuildLeft
   :- BroadcastExchange HashedRelationBroadcastMode(List(upper(input[0, string, true]))), [id=#41]
   :  +- *(1) ColumnarToRow
   :     +- FileScan parquet default.t1[c1#5,c2#6]
   +- *(2) ColumnarToRow
      +- FileScan parquet default.t2[c1#7]

== Physical Plan ==
*(2) Project [c1#5, c2#6]
+- *(2) BroadcastHashJoin [upper(c1#5)], [upper(c1#7)], Inner, BuildRight
   :- *(2) Project [c1#5, c2#6]
   :  +- *(2) Filter isnotnull(c1#5)
   :     +- *(2) ColumnarToRow
   :        +- FileScan parquet default.t1[c1#5,c2#6]
   +- BroadcastExchange HashedRelationBroadcastMode(List(upper(input[0, string, true]))), [id=#59]
      +- *(1) Project [c1#7]
         +- *(1) Filter isnotnull(c1#7)
            +- *(1) ColumnarToRow
               +- FileScan parquet default.t2[c1#7]

```

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

No.

### How was this patch tested?

Unit test.

Closes #28626 from wangyum/SPARK-28481.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-29 07:28:57 +00:00
Maryann Xue 45864faaf2 [SPARK-31862][SQL] Remove exception wrapping in AQE
### What changes were proposed in this pull request?

This PR removes the excessive exception wrapping in AQE so that error messages are less verbose and mostly consistent with non-aqe execution. Exceptions from stage materialization are now only wrapped with `SparkException` if there are multiple stage failures. Also, stage cancelling errors will not be included as part the exception thrown, but rather just be error logged.

### Why are the changes needed?

This will make the AQE error reporting more readable and debuggable.

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

No.

### How was this patch tested?

Updated existing tests.

Closes #28668 from maryannxue/spark-31862.

Authored-by: Maryann Xue <maryann.xue@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-29 04:23:38 +00:00
Maryann Xue 90302309a3 [SPARK-31865][SQL] Fix complex AQE query stage not reused
### What changes were proposed in this pull request?

This PR fixes the issue of complex query stages that contain sub stages not being reused at times due to dynamic plan changes. This PR synchronizes the "finished" flag between all reused stages so that the runtime replanning would always produce the same sub plan for their potentially reusable parent stages.

### Why are the changes needed?

Without this change, complex query stages that contain sub stages will sometimes not be reused due to dynamic plan changes and the status of their child query stages not being synced.

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

No.

### How was this patch tested?

Manually tested TPC-DS q47 and q57. Before this PR, the reuse of the biggest stage would happen with a 50/50 chance; and after this PR, it will happen 100% of the time.

Closes #28670 from maryannxue/fix-aqe-reuse.

Authored-by: Maryann Xue <maryann.xue@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-29 04:20:22 +00:00
Ali Afroozeh f6f1e51072 [SPARK-31719][SQL] Refactor JoinSelection
### What changes were proposed in this pull request?
This PR extracts the logic for selecting the planned join type out of the `JoinSelection` rule and moves it to `JoinSelectionHelper` in Catalyst.

### Why are the changes needed?
This change both cleans up the code in `JoinSelection` and allows the logic to be in one place and be used from other rules that need to make decision based on the join type before the planning time.

### Does this PR introduce _any_ user-facing change?
`BuildSide`, `BuildLeft`, and `BuildRight` are moved from `org.apache.spark.sql.execution` to Catalyst in `org.apache.spark.sql.catalyst.optimizer`.

### How was this patch tested?
This is a refactoring, passes existing tests.

Closes #28540 from dbaliafroozeh/RefactorJoinSelection.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-27 15:49:08 +00:00
iRakson 765105b6f1 [SPARK-31638][WEBUI] Clean Pagination code for all webUI pages
### What changes were proposed in this pull request?

Pagination code across pages needs to be cleaned.
I have tried to clear out these things :
* Unused methods
* Unused method arguments
* remove redundant `if` expressions
* fix indentation

### Why are the changes needed?
This fix will make code more readable and remove unnecessary methods and variables.

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

### How was this patch tested?
Manually

Closes #28448 from iRakson/refactorPagination.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-05-27 08:59:08 -05:00
beliefer 8f2b6f3a0b [SPARK-31393][SQL][FOLLOW-UP] Show the correct alias in schema for expression
### What changes were proposed in this pull request?
Some alias of expression can not display correctly in schema. This PR will fix them.
- `ln`
- `rint`
- `lcase`
- `position`

### Why are the changes needed?
Improve the implement of some expression.

### Does this PR introduce _any_ user-facing change?
 'Yes'. This PR will let user see the correct alias in schema.

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

Closes #28551 from beliefer/show-correct-alias-in-schema.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-27 15:05:06 +09:00
Max Gekk 87d34e6b96 [SPARK-31820][SQL][TESTS] Fix flaky JavaBeanDeserializationSuite
### What changes were proposed in this pull request?
Modified formatting of expected timestamp strings in the test `JavaBeanDeserializationSuite`.`testSpark22000` to correctly format timestamps with **zero** seconds fraction. Current implementation outputs `.0` but must be empty string. From SPARK-31820 failure:
- should be `2020-05-25 12:39:17`
- but incorrect expected string is `2020-05-25 12:39:17.0`

### Why are the changes needed?
To make `JavaBeanDeserializationSuite` stable, and avoid test failures like https://github.com/apache/spark/pull/28630#issuecomment-633695723

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

### How was this patch tested?
I changed 7dff3b125d/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java (L207) to
```java
new java.sql.Timestamp((System.currentTimeMillis() / 1000) * 1000),
```
to force zero seconds fraction.

Closes #28639 from MaxGekk/fix-JavaBeanDeserializationSuite.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-26 12:13:28 +00:00
Dilip Biswal b44acee953 [SPARK-31673][SQL] QueryExection.debug.toFile() to take an addtional explain mode param
### What changes were proposed in this pull request?
Currently QueryExecution.debug.toFile dumps the query plan information in a fixed format. This PR adds an additional explain mode parameter that writes the debug information as per the user supplied format.
```
df.queryExecution.debug.toFile("/tmp/plan.txt", explainMode = ExplainMode.fromString("formatted"))
```
```
== Physical Plan ==
* Filter (2)
+- Scan hive default.s1 (1)

(1) Scan hive default.s1
Output [2]: [c1#15, c2#16]
Arguments: [c1#15, c2#16], HiveTableRelation `default`.`s1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c1#15, c2#16]

(2) Filter [codegen id : 1]
Input [2]: [c1#15, c2#16]
Condition : (isnotnull(c1#15) AND (c1#15 > 0))

== Whole Stage Codegen ==
Found 1 WholeStageCodegen subtrees.
== Subtree 1 / 1 (maxMethodCodeSize:220; maxConstantPoolSize:105(0.16% used); numInnerClasses:0) ==
*(1) Filter (isnotnull(c1#15) AND (c1#15 > 0))
+- Scan hive default.s1 [c1#15, c2#16], HiveTableRelation `default`.`s1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c1#15, c2#16]

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=1
/* 006 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator inputadapter_input_0;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] filter_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1];
/* 011 */
/* 012 */   public GeneratedIteratorForCodegenStage1(Object[] references) {
/* 013 */     this.references = references;
/* 014 */   }
/* 015 */
/* 016 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 017 */     partitionIndex = index;
/* 018 */     this.inputs = inputs;
/* 019 */     inputadapter_input_0 = inputs[0];
/* 020 */     filter_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 021 */
/* 022 */   }
/* 023 */
/* 024 */   protected void processNext() throws java.io.IOException {
/* 025 */     while ( inputadapter_input_0.hasNext()) {
/* 026 */       InternalRow inputadapter_row_0 = (InternalRow) inputadapter_input_0.next();
/* 027 */
/* 028 */       do {
/* 029 */         boolean inputadapter_isNull_0 = inputadapter_row_0.isNullAt(0);
/* 030 */         int inputadapter_value_0 = inputadapter_isNull_0 ?
/* 031 */         -1 : (inputadapter_row_0.getInt(0));
/* 032 */
/* 033 */         boolean filter_value_2 = !inputadapter_isNull_0;
/* 034 */         if (!filter_value_2) continue;
/* 035 */
/* 036 */         boolean filter_value_3 = false;
/* 037 */         filter_value_3 = inputadapter_value_0 > 0;
/* 038 */         if (!filter_value_3) continue;
/* 039 */
/* 040 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 041 */
/* 042 */         boolean inputadapter_isNull_1 = inputadapter_row_0.isNullAt(1);
/* 043 */         int inputadapter_value_1 = inputadapter_isNull_1 ?
/* 044 */         -1 : (inputadapter_row_0.getInt(1));
/* 045 */         filter_mutableStateArray_0[0].reset();
/* 046 */
/* 047 */         filter_mutableStateArray_0[0].zeroOutNullBytes();
/* 048 */
/* 049 */         filter_mutableStateArray_0[0].write(0, inputadapter_value_0);
/* 050 */
/* 051 */         if (inputadapter_isNull_1) {
/* 052 */           filter_mutableStateArray_0[0].setNullAt(1);
/* 053 */         } else {
/* 054 */           filter_mutableStateArray_0[0].write(1, inputadapter_value_1);
/* 055 */         }
/* 056 */         append((filter_mutableStateArray_0[0].getRow()));
/* 057 */
/* 058 */       } while(false);
/* 059 */       if (shouldStop()) return;
/* 060 */     }
/* 061 */   }
/* 062 */
/* 063 */ }
```
### Why are the changes needed?
Hopefully enhances the usability of debug.toFile(..)

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

### How was this patch tested?
Added a test in QueryExecutionSuite

Closes #28493 from dilipbiswal/write_to_file.

Authored-by: Dilip Biswal <dkbiswal@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-26 14:40:58 +09:00
Max Gekk 7e4f5bbd8a [SPARK-31806][SQL][TESTS] Check reading date/timestamp from legacy parquet: dictionary encoding, w/o Spark version
### What changes were proposed in this pull request?
1. Add the following parquet files to the resource folder `sql/core/src/test/resources/test-data`:
   - Files saved by Spark 2.4.5 (cee4ecbb16) without meta info `org.apache.spark.version`
      - `before_1582_date_v2_4_5.snappy.parquet` with 2 date columns of the type **INT32 L:DATE** - `PLAIN` (8 date values of `1001-01-01`) and `PLAIN_DICTIONARY` (`1001-01-01`..`1001-01-08`).
      - `before_1582_timestamp_micros_v2_4_5.snappy.parquet` with 2 timestamp columns of the type **INT64 L:TIMESTAMP(MICROS,true)** - `PLAIN` (8 date values of `1001-01-01 01:02:03.123456`) and `PLAIN_DICTIONARY` (`1001-01-01 01:02:03.123456`..`1001-01-08 01:02:03.123456`).
      - `before_1582_timestamp_millis_v2_4_5.snappy.parquet` with 2 timestamp columns of the type **INT64 L:TIMESTAMP(MILLIS,true)** - `PLAIN` (8 date values of `1001-01-01 01:02:03.123`) and `PLAIN_DICTIONARY` (`1001-01-01 01:02:03.123`..`1001-01-08 01:02:03.123`).
      - `before_1582_timestamp_int96_plain_v2_4_5.snappy.parquet` with 2 timestamp columns of the type **INT96** - `PLAIN` (8 date values of `1001-01-01 01:02:03.123456`) and `PLAIN` (`1001-01-01 01:02:03.123456`..`1001-01-08 01:02:03.123456`).
      - `before_1582_timestamp_int96_dict_v2_4_5.snappy.parquet` with 2 timestamp columns of the type **INT96** - `PLAIN_DICTIONARY` (8 date values of `1001-01-01 01:02:03.123456`) and `PLAIN_DICTIONARY` (`1001-01-01 01:02:03.123456`..`1001-01-08 01:02:03.123456`).
    - Files saved by Spark 2.4.6-rc3 (570848da7c) with the meta info `org.apache.spark.version = 2.4.6`:
      - `before_1582_date_v2_4_6.snappy.parquet` replaces `before_1582_date_v2_4.snappy.parquet`. And it is similar to `before_1582_date_v2_4_5.snappy.parquet` except Spark version in parquet meta info.
      - `before_1582_timestamp_micros_v2_4_6.snappy.parquet` replaces `before_1582_timestamp_micros_v2_4.snappy.parquet`. And it is similar to `before_1582_timestamp_micros_v2_4_5.snappy.parquet` except meta info.
      - `before_1582_timestamp_millis_v2_4_6.snappy.parquet` replaces `before_1582_timestamp_millis_v2_4.snappy.parquet`. And it is similar to `before_1582_timestamp_millis_v2_4_5.snappy.parquet` except meta info.
      - `before_1582_timestamp_int96_plain_v2_4_6.snappy.parquet` is similar to `before_1582_timestamp_int96_dict_v2_4_5.snappy.parquet` except meta info.
      - `before_1582_timestamp_int96_dict_v2_4_6.snappy.parquet` replaces `before_1582_timestamp_int96_v2_4.snappy.parquet`. And it is similar to `before_1582_timestamp_int96_dict_v2_4_5.snappy.parquet` except meta info.
2. Add new test "generate test files for checking compatibility with Spark 2.4" to `ParquetIOSuite` (marked as ignored). The parquet files above were generated by this test.
3. Modified "SPARK-31159: compatibility with Spark 2.4 in reading dates/timestamps" in `ParquetIOSuite` to use new parquet files.

### Why are the changes needed?
To improve test coverage.

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

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

Closes #28630 from MaxGekk/parquet-files-update.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-26 05:15:51 +00:00
Prakhar Jain 452594f5a4 [SPARK-31810][TEST] Fix AlterTableRecoverPartitions test using incorrect api to modify RDD_PARALLEL_LISTING_THRESHOLD
### What changes were proposed in this pull request?
Use the correct API in AlterTableRecoverPartition tests to modify the `RDD_PARALLEL_LISTING_THRESHOLD` conf.

### Why are the changes needed?
The existing AlterTableRecoverPartitions test modify the RDD_PARALLEL_LISTING_THRESHOLD as a SQLConf using the withSQLConf API. But since, this is not a SQLConf, it is not overridden and so the test doesn't end up testing the required behaviour.

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

### How was this patch tested?
This is UT Fix. UTs are still passing after the fix.

Closes #28634 from prakharjain09/SPARK-31810-fix-recover-partitions.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-26 14:13:02 +09:00
HyukjinKwon df2a1fe131
[SPARK-31808][SQL] Makes struct function's output name and class name pretty
### What changes were proposed in this pull request?

This PR proposes to set the alias, and class name in its `ExpressionInfo` for `struct`.
- Class name in `ExpressionInfo`
  - from: `org.apache.spark.sql.catalyst.expressions.NamedStruct`
  - to:`org.apache.spark.sql.catalyst.expressions.CreateNamedStruct`
- Alias name: `named_struct(col1, v, ...)` -> `struct(v, ...)`

This PR takes over https://github.com/apache/spark/pull/28631

### Why are the changes needed?

To show the correct output name and class names to users.

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

Yes.

**Before:**

```scala
scala> sql("DESC FUNCTION struct").show(false)
+------------------------------------------------------------------------------------+
|function_desc                                                                       |
+------------------------------------------------------------------------------------+
|Function: struct                                                                    |
|Class: org.apache.spark.sql.catalyst.expressions.NamedStruct                        |
|Usage: struct(col1, col2, col3, ...) - Creates a struct with the given field values.|
+------------------------------------------------------------------------------------+
```

```scala
scala> sql("SELECT struct(1, 2)").show(false)
+------------------------------+
|named_struct(col1, 1, col2, 2)|
+------------------------------+
|[1, 2]                        |
+------------------------------+
```

**After:**

```scala
scala> sql("DESC FUNCTION struct").show(false)
+------------------------------------------------------------------------------------+
|function_desc                                                                       |
+------------------------------------------------------------------------------------+
|Function: struct                                                                    |
|Class: org.apache.spark.sql.catalyst.expressions.CreateNamedStruct                  |
|Usage: struct(col1, col2, col3, ...) - Creates a struct with the given field values.|
+------------------------------------------------------------------------------------+
```

```scala
scala> sql("SELECT struct(1, 2)").show(false)
+------------+
|struct(1, 2)|
+------------+
|[1, 2]      |
+------------+
```

### How was this patch tested?

Manually tested, and Jenkins tests.

Closes #28633 from HyukjinKwon/SPARK-31808.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-25 20:36:00 -07:00
Max Gekk 6c80ebbccb
[SPARK-31818][SQL] Fix pushing down filters with java.time.Instant values in ORC
### What changes were proposed in this pull request?
Convert `java.time.Instant` to `java.sql.Timestamp` in pushed down filters to ORC datasource when Java 8 time API enabled.

### Why are the changes needed?
The changes fix the exception raised while pushing date filters when `spark.sql.datetime.java8API.enabled` is set to `true`:
```
java.lang.IllegalArgumentException: Wrong value class java.time.Instant for TIMESTAMP.EQUALS leaf
 at org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl$PredicateLeafImpl.checkLiteralType(SearchArgumentImpl.java:192)
 at org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl$PredicateLeafImpl.<init>(SearchArgumentImpl.java:75)
```

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

### How was this patch tested?
Added tests to `OrcFilterSuite`.

Closes #28636 from MaxGekk/orc-timestamp-filter-pushdown.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-25 18:36:02 -07:00
Kent Yao 695cb617d4 [SPARK-31771][SQL] Disable Narrow TextStyle for datetime pattern 'G/M/L/E/u/Q/q'
### What changes were proposed in this pull request?

Five continuous pattern characters with 'G/M/L/E/u/Q/q' means Narrow-Text Style while we turn to use `java.time.DateTimeFormatterBuilder` since 3.0.0, which output the leading single letter of the value, e.g. `December` would be `D`. In Spark 2.4 they mean Full-Text Style.

In this PR, we explicitly disable Narrow-Text Style for these pattern characters.

### Why are the changes needed?

Without this change, there will be a silent data change.

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

Yes, queries with datetime operations using datetime patterns, e.g. `G/M/L/E/u` will fail if the pattern length is 5 and other patterns, e,g. 'k', 'm' also can accept a certain number of letters.

1. datetime patterns that are not supported by the new parser but the legacy will get SparkUpgradeException, e.g. "GGGGG", "MMMMM", "LLLLL", "EEEEE", "uuuuu", "aa", "aaa". 2 options are given to end-users, one is to use legacy mode, and the other is to follow the new online doc for correct datetime patterns

2, datetime patterns that are not supported by both the new parser and the legacy, e.g.  "QQQQQ", "qqqqq",  will get IllegalArgumentException which is captured by Spark internally and results NULL to end-users.

### How was this patch tested?

add unit tests

Closes #28592 from yaooqinn/SPARK-31771.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-25 15:07:41 +00:00
Max Gekk 92685c0148 [SPARK-31755][SQL][FOLLOWUP] Update date-time, CSV and JSON benchmark results
### What changes were proposed in this pull request?
Re-generate results of:
- DateTimeBenchmark
- CSVBenchmark
- JsonBenchmark

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 |

### Why are the changes needed?
1. The PR https://github.com/apache/spark/pull/28576 changed date-time parser. The `DateTimeBenchmark` should confirm that the PR didn't slow down date/timestamp parsing.
2. CSV/JSON datasources are affected by the above PR too. This PR updates the benchmark results in the same environment as other benchmarks to have a base line for future optimizations.

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

### How was this patch tested?
By running benchmarks via the script:
```python
#!/usr/bin/env python3

import os
from sparktestsupport.shellutils import run_cmd

benchmarks = [
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.DateTimeBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.datasources.csv.CSVBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.datasources.json.JsonBenchmark']
]

print('Set SPARK_GENERATE_BENCHMARK_FILES=1')
os.environ['SPARK_GENERATE_BENCHMARK_FILES'] = '1'

for b in benchmarks:
    print("Run benchmark: %s" % b[1])
    run_cmd(['build/sbt', '%s:runMain %s' % (b[0], b[1])])
```

Closes #28613 from MaxGekk/missing-hour-year-benchmarks.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-25 15:00:11 +00:00
Kent Yao 0df8dd6073 [SPARK-30352][SQL] DataSourceV2: Add CURRENT_CATALOG function
### What changes were proposed in this pull request?

As we support multiple catalogs with DataSourceV2, we may need the `CURRENT_CATALOG` value expression from the SQL standard.

`CURRENT_CATALOG` is a general value specification in the SQL Standard, described as:

> The value specified by CURRENT_CATALOG is the character string that represents the current default catalog name.

### Why are the changes needed?
improve catalog v2 with ANSI SQL standard.

### Does this PR introduce any user-facing change?
yes, add a new function `current_catalog()` to point the current active catalog

### How was this patch tested?

add ut

Closes #27006 from yaooqinn/SPARK-30352.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-25 14:27:47 +00:00
rishi b90e10c546 [SPARK-31377][SQL][TEST] Added unit tests to 'number of output rows metric' for some joins in SQLMetricSuite
### What changes were proposed in this pull request?
Add unit tests to the 'number of output rows metric' for some join types in the SQLMetricSuite. A list of unit tests added are as follows.
- ShuffledHashJoin: leftOuter, RightOuter, LeftAnti, LeftSemi
- BroadcastNestedLoopJoin: RightOuter
- BroadcastHashJoin: LeftAnti

### Why are the changes needed?
For some combinations of JoinType and Join algorithm there is no test coverage for the 'number of output rows' metric.

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

### How was this patch tested?
I added debug statements in the code to ensure the correct combination if JoinType and Join algorithms are triggered.
I further used Intellij debugger to test the same.

Closes #28330 from sririshindra/SPARK-31377.

Authored-by: rishi <spothireddi@cloudera.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-25 12:44:14 +09:00
sandeep katta cf7463f309 [SPARK-31761][SQL] cast integer to Long to avoid IntegerOverflow for IntegralDivide operator
### What changes were proposed in this pull request?
`IntegralDivide` operator returns Long DataType, so integer overflow case should be handled.
If the operands are of type Int it will be casted to Long

### Why are the changes needed?
As `IntegralDivide` returns Long datatype, integer overflow should not happen

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

### How was this patch tested?
Added UT and also tested in the local cluster

After fix

![image](https://user-images.githubusercontent.com/35216143/82603361-25eccc00-9bd0-11ea-9ca7-001c539e628b.png)

SQL Test

After fix
![image](https://user-images.githubusercontent.com/35216143/82637689-f0250300-9c22-11ea-85c3-886ab2c23471.png)

Before Fix
![image](https://user-images.githubusercontent.com/35216143/82637984-878a5600-9c23-11ea-9e47-5ce2fb923c01.png)

Closes #28600 from sandeep-katta/integerOverFlow.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-24 14:50:11 +09:00
Gengliang Wang 9fdc2a0801 [SPARK-31793][SQL] Reduce the memory usage in file scan location metadata
### What changes were proposed in this pull request?

Currently, the data source scan node stores all the paths in its metadata. The metadata is kept when a SparkPlan is converted into SparkPlanInfo. SparkPlanInfo can be used to construct the Spark plan graph in UI.

However, the paths can be very large (e.g. it can be many partitions after partition pruning), while UI pages only require up to 100 bytes for the location metadata. We can reduce the paths stored in metadata to reduce memory usage.

### Why are the changes needed?

Reduce unnecessary memory cost.
In the heap dump of a driver, the SparkPlanInfo instances are quite large and it should be avoided:
![image](https://user-images.githubusercontent.com/1097932/82642318-8f65de00-9bc2-11ea-9c9c-f05c2b0e1c49.png)

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

No

### How was this patch tested?

Unit tests

Closes #28610 from gengliangwang/improveLocationMetadata.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-05-23 15:00:28 -07:00
iRakson fbb3144a9c [SPARK-31642] Add Pagination Support for Structured Streaming Page
### What changes were proposed in this pull request?
Add Pagination Support for structured streaming page. Now both tables `Active Queries` and `Completed Queries` will have pagination.
To implement pagination, pagination framework from #7399  is used.
* Also tables will only be shown if there is at least one entry in the table.

### Why are the changes needed?
* This will help users in analysing their structured streaming queries in much better way.
* Other Web UI pages support pagination in their table. So this will make web UI more consistent across pages.
* This can prevent potential OOM errors.

### Does this PR introduce _any_ user-facing change?
Yes. Both tables will support pagination.

### How was this patch tested?
Manually. I will add snapshots soon.

Closes #28485 from iRakson/SPARK-31642.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-05-23 17:17:53 +09:00
Takeshi Yamamuro 7ca73f03fb [SPARK-29854][SQL][TESTS] Add tests to check lpad/rpad throw an exception for invalid length input
### What changes were proposed in this pull request?

This PR intends to add trivial tests to check https://github.com/apache/spark/pull/27024 has already been fixed in the master.

Closes #27024

### Why are the changes needed?

For test coverage.

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

No.

### How was this patch tested?

Added tests.

Closes #28604 from maropu/SPARK-29854.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-05-23 08:48:29 +09:00
Jungtaek Lim (HeartSaVioR) 5a258b0b67
[SPARK-30915][SS] CompactibleFileStreamLog: Avoid reading the metadata log file when finding the latest batch ID
### What changes were proposed in this pull request?

This patch adds the new method `getLatestBatchId()` in CompactibleFileStreamLog in complement of getLatest() which doesn't read the content of the latest batch metadata log file, and apply to both FileStreamSource and FileStreamSink to avoid unnecessary latency on reading log file.

### Why are the changes needed?

Once compacted metadata log file becomes huge, writing outputs for the compact + 1 batch is also affected due to unnecessarily reading the compacted metadata log file. This unnecessary latency can be simply avoided.

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

No.

### How was this patch tested?

New UT. Also manually tested under query which has huge metadata log on file stream sink:

> before applying the patch

![Screen Shot 2020-02-21 at 4 20 19 PM](https://user-images.githubusercontent.com/1317309/75016223-d3ffb180-54cd-11ea-9063-49405943049d.png)

> after applying the patch

![Screen Shot 2020-02-21 at 4 06 18 PM](https://user-images.githubusercontent.com/1317309/75016220-d235ee00-54cd-11ea-81a7-7c03a43c4db4.png)

Peaks are compact batches - please compare the next batch after compact batches, especially the area of "light brown".

Closes #27664 from HeartSaVioR/SPARK-30915.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-05-22 16:46:17 -07:00
TJX2014 2115c55efe [SPARK-31710][SQL] Adds TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions
### What changes were proposed in this pull request?
Add and register three new functions: `TIMESTAMP_SECONDS`, `TIMESTAMP_MILLIS` and `TIMESTAMP_MICROS`
A test is added.

Reference: [BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/timestamp_functions?hl=en#timestamp_seconds)

### Why are the changes needed?
People will have convenient way to get timestamps from seconds,milliseconds and microseconds.

### Does this PR introduce _any_ user-facing change?
Yes, people will have the following ways to get timestamp:

```scala
sql("select TIMESTAMP_SECONDS(t.a) as timestamp from values(1230219000),(-1230219000) as t(a)").show(false)
```
```
+-------------------------+
|timestamp                  |
+-------------------------+
|2008-12-25 23:30:00|
|1931-01-07 16:30:00|
+-------------------------+
```
```scala
sql("select TIMESTAMP_MILLIS(t.a) as timestamp from values(1230219000123),(-1230219000123) as t(a)").show(false)
```
```
+-------------------------------+
|timestamp                           |
+-------------------------------+
|2008-12-25 23:30:00.123|
|1931-01-07 16:29:59.877|
+-------------------------------+
```
```scala
sql("select TIMESTAMP_MICROS(t.a) as timestamp from values(1230219000123123),(-1230219000123123) as t(a)").show(false)
```
```
+------------------------------------+
|timestamp                                   |
+------------------------------------+
|2008-12-25 23:30:00.123123|
|1931-01-07 16:29:59.876877|
+------------------------------------+
```
### How was this patch tested?
Unit test.

Closes #28534 from TJX2014/master-SPARK-31710.

Authored-by: TJX2014 <xiaoxingstack@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-22 14:16:30 +00:00
Wenchen Fan ce4da29ec3 [SPARK-31755][SQL] allow missing year/hour when parsing date/timestamp string
### What changes were proposed in this pull request?

This PR allows missing hour fields when parsing date/timestamp string, with 0 as the default value.

If the year field is missing, this PR still fail the query by default, but provides a new legacy config to allow it and use 1970 as the default value. It's not a good default value, as it is not a leap year, which means that it would never parse Feb 29. We just pick it for backward compatibility.

### Why are the changes needed?

To keep backward compatibility with Spark 2.4.

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

Yes.

Spark 2.4:
```
scala> sql("select to_timestamp('16', 'dd')").show
+------------------------+
|to_timestamp('16', 'dd')|
+------------------------+
|     1970-01-16 00:00:00|
+------------------------+

scala> sql("select to_date('16', 'dd')").show
+-------------------+
|to_date('16', 'dd')|
+-------------------+
|         1970-01-16|
+-------------------+

scala> sql("select to_timestamp('2019 40', 'yyyy mm')").show
+----------------------------------+
|to_timestamp('2019 40', 'yyyy mm')|
+----------------------------------+
|               2019-01-01 00:40:00|
+----------------------------------+

scala> sql("select to_timestamp('2019 10:10:10', 'yyyy hh:mm:ss')").show
+----------------------------------------------+
|to_timestamp('2019 10:10:10', 'yyyy hh:mm:ss')|
+----------------------------------------------+
|                           2019-01-01 10:10:10|
+----------------------------------------------+
```

in branch 3.0
```
scala> sql("select to_timestamp('16', 'dd')").show
+--------------------+
|to_timestamp(16, dd)|
+--------------------+
|                null|
+--------------------+

scala> sql("select to_date('16', 'dd')").show
+---------------+
|to_date(16, dd)|
+---------------+
|           null|
+---------------+

scala> sql("select to_timestamp('2019 40', 'yyyy mm')").show
+------------------------------+
|to_timestamp(2019 40, yyyy mm)|
+------------------------------+
|           2019-01-01 00:00:00|
+------------------------------+

scala> sql("select to_timestamp('2019 10:10:10', 'yyyy hh:mm:ss')").show
+------------------------------------------+
|to_timestamp(2019 10:10:10, yyyy hh:mm:ss)|
+------------------------------------------+
|                       2019-01-01 00:00:00|
+------------------------------------------+
```

After this PR, the behavior becomes the same as 2.4, if the legacy config is enabled.

### How was this patch tested?

new tests

Closes #28576 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-22 16:10:08 +09:00
Max Gekk 60118a2426 [SPARK-31785][SQL][TESTS] Add a helper function to test all parquet readers
### What changes were proposed in this pull request?
Add `withAllParquetReaders` to `ParquetTest`. The function allow to run a block of code for all available Parquet readers.

### Why are the changes needed?
1. It simplifies tests
2. Allow to test all parquet readers that could be available in projects based on Apache Spark.

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

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

Closes #28598 from MaxGekk/add-withAllParquetReaders.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-22 09:53:35 +09:00
Gengliang Wang db5e5fce68 Revert "[SPARK-31765][WEBUI] Upgrade HtmlUnit >= 2.37.0"
This reverts commit 92877c4ef2.

Closes #28602 from gengliangwang/revertSPARK-31765.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-05-21 16:00:58 -07:00
Kousuke Saruta 92877c4ef2 [SPARK-31765][WEBUI] Upgrade HtmlUnit >= 2.37.0
### What changes were proposed in this pull request?

This PR upgrades HtmlUnit.
Selenium and Jetty also upgraded because of dependency.
### Why are the changes needed?

Recently, a security issue which affects HtmlUnit is reported.
https://nvd.nist.gov/vuln/detail/CVE-2020-5529
According to the report, arbitrary code can be run by malicious users.
HtmlUnit is used for test so the impact might not be large but it's better to upgrade it just in case.

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

No.

### How was this patch tested?

Existing testcases.

Closes #28585 from sarutak/upgrade-htmlunit.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-05-21 11:43:25 -07:00
iRakson f1495c5bc0 [SPARK-31688][WEBUI] Refactor Pagination framework
### What changes were proposed in this pull request?
Currently while implementing pagination using the existing pagination framework, a lot of code is being copied as pointed out [here](https://github.com/apache/spark/pull/28485#pullrequestreview-408881656).

I introduced some changes in `PagedTable` which is the main trait for implementing the pagination.
* Added function for getting table parameters.
* Added a function for table header row. This will help in maintaining consistency across the tables. All the header rows across tables will be consistent now.

### Why are the changes needed?

* A lot of code is copied every time pagination is implemented for any table.
* Code readability is not great as lot of HTML is embedded.
* Paginating other tables will be a lot easier now.

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

### How was this patch tested?
Manually. This is mainly refactoring work, no new functionality introduced. Existing test cases should pass.

Closes #28512 from iRakson/refactorPaginationFramework.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-05-21 13:00:00 -05:00
Vinoo Ganesh dae79888dc [SPARK-31354] SparkContext only register one SparkSession ApplicationEnd listener
## What changes were proposed in this pull request?

This change was made as a result of the conversation on https://issues.apache.org/jira/browse/SPARK-31354 and is intended to continue work from that ticket here.

This change fixes a memory leak where SparkSession listeners are never cleared off of the SparkContext listener bus.

Before running this PR, the following code:
```
SparkSession.builder().master("local").getOrCreate()
SparkSession.clearActiveSession()
SparkSession.clearDefaultSession()

SparkSession.builder().master("local").getOrCreate()
SparkSession.clearActiveSession()
SparkSession.clearDefaultSession()
```
would result in a SparkContext with the following listeners on the listener bus:
```
[org.apache.spark.status.AppStatusListener5f610071,
org.apache.spark.HeartbeatReceiverd400c17,
org.apache.spark.sql.SparkSession$$anon$125849aeb, <-First instance
org.apache.spark.sql.SparkSession$$anon$1fadb9a0] <- Second instance
```
After this PR, the execution of the same code above results in SparkContext with the following listeners on the listener bus:
```
[org.apache.spark.status.AppStatusListener5f610071,
org.apache.spark.HeartbeatReceiverd400c17,
org.apache.spark.sql.SparkSession$$anon$125849aeb] <-One instance
```
## How was this patch tested?

* Unit test included as a part of the PR

Closes #28128 from vinooganesh/vinooganesh/SPARK-27958.

Lead-authored-by: Vinoo Ganesh <vinoo.ganesh@gmail.com>
Co-authored-by: Vinoo Ganesh <vganesh@palantir.com>
Co-authored-by: Vinoo Ganesh <vinoo@safegraph.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-21 16:06:28 +00:00
Max Gekk 5d673319af [SPARK-31762][SQL] Fix perf regression of date/timestamp formatting in toHiveString
### What changes were proposed in this pull request?
1. Add new methods that accept date-time Java types to the DateFormatter and TimestampFormatter traits. The methods format input date-time instances to strings:
    - TimestampFormatter:
      - `def format(ts: Timestamp): String`
      - `def format(instant: Instant): String`
    - DateFormatter:
      - `def format(date: Date): String`
      - `def format(localDate: LocalDate): String`
2. Re-use the added methods from `HiveResult.toHiveString`
3. Borrow the code for formatting of `java.sql.Timestamp` from Spark 2.4 `DateTimeUtils.timestampToString` to `FractionTimestampFormatter` because legacy formatters don't support variable length patterns for seconds fractions.

### Why are the changes needed?
To avoid unnecessary overhead of converting Java date-time types to micros/days before formatting. Also formatters have to convert input micros/days back to Java types to pass instances to standard library API.

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

### How was this patch tested?
By existing tests for toHiveString and new tests in `TimestampFormatterSuite`.

Closes #28582 from MaxGekk/opt-format-old-types.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-21 04:01:19 +00:00
Wenchen Fan 34414acfa3 [SPARK-31706][SQL] add back the support of streaming update mode
### What changes were proposed in this pull request?

This PR adds a private `WriteBuilder` mixin trait: `SupportsStreamingUpdate`, so that the builtin v2 streaming sinks can still support the update mode.

Note: it's private because we don't have a proper design yet. I didn't take the proposal in https://github.com/apache/spark/pull/23702#discussion_r258593059 because we may want something more general, like updating by an expression `key1 = key2 + 10`.

### Why are the changes needed?

In Spark 2.4, all builtin v2 streaming sinks support all streaming output modes, and v2 sinks are enabled by default, see https://issues.apache.org/jira/browse/SPARK-22911

It's too risky for 3.0 to go back to v1 sinks, so I propose to add a private trait to fix builtin v2 sinks, to keep backward compatibility.

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

Yes, now all the builtin v2 streaming sinks support all streaming output modes, which is the same as 2.4

### How was this patch tested?

existing tests.

Closes #28523 from cloud-fan/update.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-20 03:45:13 +00:00
yi.wu 0fd98abd85 [SPARK-31750][SQL] Eliminate UpCast if child's dataType is DecimalType
### What changes were proposed in this pull request?

Eliminate the `UpCast` if it's child data type is already decimal type.

### Why are the changes needed?

While deserializing internal `Decimal` value to external `BigDecimal`(Java/Scala) value, Spark should also respect `Decimal`'s precision and scale, otherwise it will cause precision lost and look weird in some cases, e.g.:

```
sql("select cast(11111111111111111111111111111111111111 as decimal(38, 0)) as d")
  .write.mode("overwrite")
  .parquet(f.getAbsolutePath)

// can fail
spark.read.parquet(f.getAbsolutePath).as[BigDecimal]
```
```
[info]   org.apache.spark.sql.AnalysisException: Cannot up cast `d` from decimal(38,0) to decimal(38,18).
[info] The type path of the target object is:
[info] - root class: "scala.math.BigDecimal"
[info] You can either add an explicit cast to the input data or choose a higher precision type of the field in the target object;
[info]   at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveUpCast$.org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveUpCast$$fail(Analyzer.scala:3060)
[info]   at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveUpCast$$anonfun$apply$33$$anonfun$applyOrElse$174.applyOrElse(Analyzer.scala:3087)
[info]   at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveUpCast$$anonfun$apply$33$$anonfun$applyOrElse$174.applyOrElse(Analyzer.scala:3071)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:309)
[info]   at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:309)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:314)
```

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

Yes, for cases(cause precision lost) mentioned above will fail before this change but run successfully after this change.

### How was this patch tested?

Added tests.

Closes #28572 from Ngone51/fix_encoder.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-20 11:00:58 +09:00
Ali Afroozeh b9cc31cd95 [SPARK-31721][SQL] Assert optimized is initialized before tracking the planning time
### What changes were proposed in this pull request?
The QueryPlanningTracker in QueryExeuction reports the planning time that also includes the optimization time. This happens because the optimizedPlan in QueryExecution is lazy and only will initialize when first called. When df.queryExecution.executedPlan is called, the the tracker starts recording the planning time, and then calls the optimized plan. This causes the planning time to start before optimization and also include the planning time.
This PR fixes this behavior by introducing a method assertOptimized, similar to assertAnalyzed that explicitly initializes the optimized plan. This method is called before measuring the time for sparkPlan and executedPlan. We call it before sparkPlan because that also counts as planning time.

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

### How was this patch tested?
Unit tests

Closes #28543 from dbaliafroozeh/AddAssertOptimized.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2020-05-19 11:10:49 +02:00
Eren Avsarogullari ab4cf49a1c [SPARK-31440][SQL] Improve SQL Rest API
### What changes were proposed in this pull request?
SQL Rest API exposes query execution metrics as Public API. This PR aims to apply following improvements on SQL Rest API by aligning Spark-UI.

**Proposed Improvements:**
1- Support Physical Operations and group metrics per physical operation by aligning Spark UI.
2- Support `wholeStageCodegenId` for Physical Operations
3- `nodeId` can be useful for grouping metrics and sorting physical operations (according to execution order) to differentiate same operators (if used multiple times during the same query execution) and their metrics.
4- Filter `empty` metrics by aligning with Spark UI - SQL Tab. Currently, Spark UI does not show empty metrics.
5- Remove line breakers(`\n`) from `metricValue`.
6- `planDescription` can be `optional` Http parameter to avoid network cost where there is specially complex jobs creating big-plans.
7- `metrics` attribute needs to be exposed at the bottom order as `nodes`. Specially, this can be useful for the user where `nodes` array size is high.
8- `edges` attribute is being exposed to show relationship between `nodes`.
9- Reverse order on `metricDetails` aims to match with Spark UI by supporting Physical Operators' execution order.

### Why are the changes needed?
Proposed improvements provides more useful (e.g: physical operations and metrics correlation, grouping) and clear (e.g: filtering blank metrics, removing line breakers) result for the end-user.

### Does this PR introduce any user-facing change?
Yes. Please find both current and improved versions of the results as attached for following SQL Rest Endpoint:
```
curl -X GET http://localhost:4040/api/v1/applications/$appId/sql/$executionId?details=true
```
**Current version:**
https://issues.apache.org/jira/secure/attachment/12999821/current_version.json

**Improved version:**
https://issues.apache.org/jira/secure/attachment/13000621/improved_version.json

### Backward Compatibility
SQL Rest API will be started to expose with `Spark 3.0` and `3.0.0-preview2` (released on 12/23/19) does not cover this API so if PR can catch 3.0 release, this will not have any backward compatibility issue.

### How was this patch tested?
1. New Unit tests are added.
2. Also, patch has been tested manually through both **Spark Core** and **History Server** Rest APIs.

Closes #28208 from erenavsarogullari/SPARK-31440.

Authored-by: Eren Avsarogullari <eren.avsarogullari@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-05-18 23:21:32 -07:00
Jungtaek Lim (HeartSaVioR) d2bec5e265 [SPARK-31707][SQL] Revert SPARK-30098 Use default datasource as provider for CREATE TABLE syntax
### What changes were proposed in this pull request?

This patch effectively reverts SPARK-30098 via below changes:

* Removed the config
* Removed the changes done in parser rule
* Removed the usage of config in tests
  * Removed tests which depend on the config
  * Rolled back some tests to before SPARK-30098 which were affected by SPARK-30098
* Reflect the change into docs (migration doc, create table syntax)

### Why are the changes needed?

SPARK-30098 brought confusion and frustration on using create table DDL query, and we agreed about the bad effect on the change.

Please go through the [discussion thread](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-Resolve-ambiguous-parser-rule-between-two-quot-create-table-quot-s-td29051i20.html) to see the details.

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

No, compared to Spark 2.4.x. End users tried to experiment with Spark 3.0.0 previews will see the change that the behavior is going back to Spark 2.4.x, but I believe we won't guarantee compatibility in preview releases.

### How was this patch tested?

Existing UTs.

Closes #28517 from HeartSaVioR/revert-SPARK-30098.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-17 02:27:23 +00:00
Max Gekk 5539ecfdac [SPARK-31725][CORE][SQL][TESTS] Set America/Los_Angeles time zone and Locale.US in tests by default
### What changes were proposed in this pull request?
Set default time zone and locale in the default constructor of `SparkFunSuite`:
- Default time zone to `America/Los_Angeles`
- Default locale to `Locale.US`

### Why are the changes needed?
1. To deduplicate code by moving common time zone and locale settings to one place SparkFunSuite
2. To have the same default time zone and locale in all tests. This should prevent errors like https://github.com/apache/spark/pull/28538

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

### How was this patch tested?
by running all affected test suites

Closes #28548 from MaxGekk/timezone-settings-SparkFunSuite.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-17 02:26:00 +00:00
Yuanjian Li 86bd37f37e [SPARK-31663][SQL] Grouping sets with having clause returns the wrong result
### What changes were proposed in this pull request?
- Resolve the havingcondition with expanding the GROUPING SETS/CUBE/ROLLUP expressions together in `ResolveGroupingAnalytics`:
    - Change the operations resolving directions to top-down.
    - Try resolving the condition of the filter as though it is in the aggregate clause by reusing the function in `ResolveAggregateFunctions`
    - Push the aggregate expressions into the aggregate which contains the expanded operations.
- Use UnresolvedHaving for all having clause.

### Why are the changes needed?
Correctness bug fix. See the demo and analysis in SPARK-31663.

### Does this PR introduce _any_ user-facing change?
Yes, correctness bug fix for HAVING with GROUPING SETS.

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

Closes #28501 from xuanyuanking/SPARK-31663.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-16 04:37:18 +00:00
yi.wu d8b001fa87 [SPARK-31620][SQL] Fix reference binding failure in case of an final agg contains subquery
### What changes were proposed in this pull request?

Instead of using `child.output` directly, we should use `inputAggBufferAttributes` from the current agg expression  for `Final` and `PartialMerge` aggregates to bind references for their `mergeExpression`.

### Why are the changes needed?

When planning aggregates, the partial aggregate uses agg fucs' `inputAggBufferAttributes` as its output, see https://github.com/apache/spark/blob/v3.0.0-rc1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala#L105

For final `HashAggregateExec`, we need to bind the `DeclarativeAggregate.mergeExpressions` with the output of the partial aggregate operator, see https://github.com/apache/spark/blob/v3.0.0-rc1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L348

This is usually fine. However, if we copy the agg func somehow after agg planning, like `PlanSubqueries`, the `DeclarativeAggregate` will be replaced by a new instance with new `inputAggBufferAttributes` and `mergeExpressions`. Then we can't bind the `mergeExpressions` with the output of the partial aggregate operator, as it uses the `inputAggBufferAttributes` of the original `DeclarativeAggregate` before copy.

Note that, `ImperativeAggregate` doesn't have this problem, as we don't need to bind its `mergeExpressions`. It has a different mechanism to access buffer values, via `mutableAggBufferOffset` and `inputAggBufferOffset`.

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

Yes, user hit error previously but run query successfully after this change.

### How was this patch tested?

Added a regression test.

Closes #28496 from Ngone51/spark-31620.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-15 15:36:28 +00:00
Karuppayya Rajendran 72601460ad
[SPARK-31692][SQL] Pass hadoop confs specifed via Spark confs to URLStreamHandlerfactory
### What changes were proposed in this pull request?
Pass hadoop confs  specifed via Spark confs to URLStreamHandlerfactory

### Why are the changes needed?

**BEFORE**
```
➜  spark git:(SPARK-31692) ✗ ./bin/spark-shell --conf spark.hadoop.fs.file.impl=org.apache.hadoop.fs.RawLocalFileSystem

scala> spark.sharedState
res0: org.apache.spark.sql.internal.SharedState = org.apache.spark.sql.internal.SharedState5793cd84

scala> new java.net.URL("file:///tmp/1.txt").openConnection.getInputStream
res1: java.io.InputStream = org.apache.hadoop.fs.ChecksumFileSystem$FSDataBoundedInputStream22846025

scala> import org.apache.hadoop.fs._
import org.apache.hadoop.fs._

scala>  FileSystem.get(new Path("file:///tmp/1.txt").toUri, spark.sparkContext.hadoopConfiguration)
res2: org.apache.hadoop.fs.FileSystem = org.apache.hadoop.fs.LocalFileSystem5a930c03
```

**AFTER**
```
➜  spark git:(SPARK-31692) ✗ ./bin/spark-shell --conf spark.hadoop.fs.file.impl=org.apache.hadoop.fs.RawLocalFileSystem

scala> spark.sharedState
res0: org.apache.spark.sql.internal.SharedState = org.apache.spark.sql.internal.SharedState5c24a636

scala> new java.net.URL("file:///tmp/1.txt").openConnection.getInputStream
res1: java.io.InputStream = org.apache.hadoop.fs.FSDataInputStream2ba8f528

scala> import org.apache.hadoop.fs._
import org.apache.hadoop.fs._

scala>  FileSystem.get(new Path("file:///tmp/1.txt").toUri, spark.sparkContext.hadoopConfiguration)
res2: org.apache.hadoop.fs.FileSystem = LocalFS

scala>  FileSystem.get(new Path("file:///tmp/1.txt").toUri, spark.sparkContext.hadoopConfiguration).getClass
res3: Class[_ <: org.apache.hadoop.fs.FileSystem] = class org.apache.hadoop.fs.RawLocalFileSystem
```
The type of FileSystem object created(you can check the last statement in the above snippets) in the above two cases are different, which should not have been the case

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

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

Closes #28516 from karuppayya/SPARK-31692.

Authored-by: Karuppayya Rajendran <karuppayya1990@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-13 23:18:38 -07:00
Wenchen Fan fd2d55c991 [SPARK-31405][SQL] Fail by default when reading/writing legacy datetime values from/to Parquet/Avro files
### What changes were proposed in this pull request?

When reading/writing datetime values that before the rebase switch day, from/to Avro/Parquet files, fail by default and ask users to set a config to explicitly do rebase or not.

### Why are the changes needed?

Rebase or not rebase have different behaviors and we should let users decide it explicitly. In most cases, users won't hit this exception as it only affects ancient datetime values.

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

Yes, now users will see an error when reading/writing dates before 1582-10-15 or timestamps before 1900-01-01 from/to Parquet/Avro files, with an error message to ask setting a config.

### How was this patch tested?

updated tests

Closes #28477 from cloud-fan/rebase.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-14 12:32:40 +09:00
beliefer a89006aba0 [SPARK-31393][SQL] Show the correct alias in schema for expression
### What changes were proposed in this pull request?
Some alias of expression can not display correctly in schema. This PR will fix them.
- `TimeWindow`
- `MaxBy`
- `MinBy`
- `UnaryMinus`
- `BitwiseCount`

This PR also fix a typo issue, please look at b7cde42b04/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala (L142)

Note:

1. `MaxBy` and `MinBy` extends `MaxMinBy` and the latter add a method `funcName` not needed.  We can reuse `prettyName` to replace `funcName`.
2. Spark SQL exists some function no elegant implementation.For example: `BitwiseCount` override the sql method show below:
`override def sql: String = s"bit_count(${child.sql})"`
I don't think it's elegant enough. Because `Expression` gives the following definitions.
```
  def sql: String = {
    val childrenSQL = children.map(_.sql).mkString(", ")
    s"$prettyName($childrenSQL)"
  }
```
By this definition, `BitwiseCount` should override `prettyName` method.

### Why are the changes needed?
Improve the implement of some expression.

### Does this PR introduce any user-facing change?
 'Yes'. This PR will let user see the correct alias in schema.

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

Closes #28164 from beliefer/elegant-pretty-name-for-function.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-12 10:25:04 +09:00
Gabor Somogyi 5a5af46a94 [SPARK-31575][SQL] Synchronise global JVM security configuration modification
### What changes were proposed in this pull request?
There is a race in secure JDBC connection providers. Namely multiple providers can read and/or write the exact same JVM security configuration at the same time. In this PR I've synchronised them with an object class. Since the configuration read and write takes couple of milliseconds it won't cause performance degradation.

### Why are the changes needed?
There is a race in secure JDBC connection providers.

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

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

Closes #28368 from gaborgsomogyi/SPARK-31575.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-05-11 09:10:58 -05:00
Max Gekk 5d5866be12 [SPARK-31672][SQL] Fix loading of timestamps before 1582-10-15 from dictionary encoded Parquet columns
### What changes were proposed in this pull request?
Modified the `decodeDictionaryIds()` method of `VectorizedColumnReader` to handle especially `TimestampType` when the passed parameter `rebaseDateTime` is true. In that case, decoded milliseconds/microseconds are rebased from the hybrid calendar to Proleptic Gregorian calendar using `RebaseDateTime`.`rebaseJulianToGregorianMicros()`.

### Why are the changes needed?
This fixes the bug of loading timestamps before the cutover day from dictionary encoded column in parquet files. The code below forces dictionary encoding:
```scala
spark.conf.set("spark.sql.legacy.parquet.rebaseDateTimeInWrite.enabled", true)
scala> spark.conf.set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MICROS")
scala>
Seq.tabulate(8)(_ => "1001-01-01 01:02:03.123").toDF("tsS")
  .select($"tsS".cast("timestamp").as("ts")).repartition(1)
  .write
  .option("parquet.enable.dictionary", true)
  .parquet(path)
```
Load the dates back:
```scala
scala> spark.read.parquet(path).show(false)
+-----------------------+
|ts                     |
+-----------------------+
|1001-01-07 00:32:20.123|
...
|1001-01-07 00:32:20.123|
+-----------------------+
```
Expected values **must be 1001-01-01 01:02:03.123** but not 1001-01-07 00:32:20.123.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes:
```scala
scala> spark.read.parquet(path).show(false)
+-----------------------+
|ts                     |
+-----------------------+
|1001-01-01 01:02:03.123|
...
|1001-01-01 01:02:03.123|
+-----------------------+
```

### How was this patch tested?
Modified the test `SPARK-31159: rebasing timestamps in write` in `ParquetIOSuite` to checked reading dictionary encoded dates.

Closes #28489 from MaxGekk/fix-ts-rebase-parquet-dict-enc.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-11 04:58:08 +00:00
Max Gekk ce63bef1da [SPARK-31662][SQL] Fix loading of dates before 1582-10-15 from dictionary encoded Parquet columns
### What changes were proposed in this pull request?
Modified the `decodeDictionaryIds()` method `VectorizedColumnReader` to handle especially the `DateType` when passed parameter `rebaseDateTime` is true. In that case, decoded days are rebased from the hybrid calendar to Proleptic Gregorian calendar using `RebaseDateTime`.`rebaseJulianToGregorianDays()`.

### Why are the changes needed?
This fixes the bug of loading dates before the cutover day from dictionary encoded column in parquet files. The code below forces dictionary encoding:
```scala
spark.conf.set("spark.sql.legacy.parquet.rebaseDateTimeInWrite.enabled", true)
Seq.tabulate(8)(_ => "1001-01-01").toDF("dateS")
  .select($"dateS".cast("date").as("date")).repartition(1)
  .write
  .option("parquet.enable.dictionary", true)
  .parquet(path)
```
Load the dates back:
```scala
spark.read.parquet(path).show(false)
+----------+
|date      |
+----------+
|1001-01-07|
...
|1001-01-07|
+----------+
```
Expected values **must be 1000-01-01** but not 1001-01-07.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes:
```scala
spark.read.parquet(path).show(false)
+----------+
|date      |
+----------+
|1001-01-01|
...
|1001-01-01|
+----------+
```

### How was this patch tested?
Modified the test `SPARK-31159: rebasing dates in write` in `ParquetIOSuite` to checked reading dictionary encoded dates.

Closes #28479 from MaxGekk/fix-datetime-rebase-parquet-dict-enc.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-10 13:31:26 +09:00
manuzhang 77c690a725 [SPARK-31658][SQL] Fix SQL UI not showing write commands of AQE plan
### What changes were proposed in this pull request?
Show write commands on SQL UI of an AQE plan

### Why are the changes needed?
Currently the leaf node of an AQE plan is always a `AdaptiveSparkPlan` which is not true when it's a child of a write command. Hence, the node of the write command as well as its metrics are not shown on the SQL UI.

#### Before

![image](https://user-images.githubusercontent.com/1191767/81288918-1893f580-9098-11ea-9771-e3d0820ba806.png)

#### After

![image](https://user-images.githubusercontent.com/1191767/81289008-3a8d7800-9098-11ea-93ec-516bbaf25d2d.png)

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

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

Closes #28474 from manuzhang/aqe-ui.

Lead-authored-by: manuzhang <owenzhang1990@gmail.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2020-05-08 10:24:13 -07:00
Max Gekk 272d229005 [SPARK-31361][SQL][TESTS][FOLLOWUP] Check non-vectorized Parquet reader while date/timestamp rebasing
### What changes were proposed in this pull request?
In PR, I propose to modify two tests of `ParquetIOSuite`:
- SPARK-31159: rebasing timestamps in write
- SPARK-31159: rebasing dates in write

to check non-vectorized Parquet reader together with vectorized reader.

### Why are the changes needed?
To improve test coverage and make sure that non-vectorized reader behaves similar to the vectorized reader.

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

### How was this patch tested?
By running `PaquetIOSuite`:
```
$ ./build/sbt "test:testOnly *ParquetIOSuite"
```

Closes #28466 from MaxGekk/test-novec-rebase-ParquetIOSuite.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-07 07:52:29 +00:00
Kent Yao b31ae7bb0b [SPARK-31615][SQL] Pretty string output for sql method of RuntimeReplaceable expressions
### What changes were proposed in this pull request?

The RuntimeReplaceable ones are runtime replaceable, thus, their original parameters are not going to be resolved to PrettyAttribute and remain debug style string if we directly implement their `sql` methods with their parameters' `sql` methods.

This PR is raised with suggestions by maropu and cloud-fan https://github.com/apache/spark/pull/28402/files#r417656589. In this PR, we re-implement the `sql` methods of  the RuntimeReplaceable ones with toPettySQL

### Why are the changes needed?

Consistency of schema output between RuntimeReplaceable expressions and normal ones.

For example, `date_format` vs `to_timestamp`, before this PR, they output differently

#### Before
```sql
select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uuuu')
struct<date_format(TIMESTAMP '2019-10-06 00:00:00', yyyy-MM-dd uuuu):string>

select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS")
struct<to_timestamp('2019-10-06S10:11:12.12345', 'yyyy-MM-dd\'S\'HH:mm:ss.SSSSSS'):timestamp>
```
#### After

```sql
select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uuuu')
struct<date_format(TIMESTAMP '2019-10-06 00:00:00', yyyy-MM-dd uuuu):string>

select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS")

struct<to_timestamp(2019-10-06T10:11:12'12, yyyy-MM-dd'T'HH:mm:ss''SSSS):timestamp>

````

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

Yes, the schema output style changed for the runtime replaceable expressions as shown in the above example

### How was this patch tested?
regenerate all related tests

Closes #28420 from yaooqinn/SPARK-31615.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-05-07 14:40:26 +09:00
Max Gekk 3d38bc2605 [SPARK-31361][SQL][FOLLOWUP] Use LEGACY_PARQUET_REBASE_DATETIME_IN_READ instead of avro config in ParquetIOSuite
### What changes were proposed in this pull request?
Replace the Avro SQL config `LEGACY_AVRO_REBASE_DATETIME_IN_READ ` by `LEGACY_PARQUET_REBASE_DATETIME_IN_READ ` in `ParquetIOSuite`.

### Why are the changes needed?
Avro config is not relevant to the parquet tests.

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

### How was this patch tested?
By running `ParquetIOSuite` via
```
./build/sbt "test:testOnly *ParquetIOSuite"
```

Closes #28461 from MaxGekk/fix-conf-in-ParquetIOSuite.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-07 09:46:42 +09:00
yi.wu b16ea8e1ab [SPARK-31650][SQL] Fix wrong UI in case of AdaptiveSparkPlanExec has unmanaged subqueries
### What changes were proposed in this pull request?

Make the non-subquery `AdaptiveSparkPlanExec` update UI again after execute/executeCollect/executeTake/executeTail if the `AdaptiveSparkPlanExec` has subqueries which do not belong to any query stages.

### Why are the changes needed?

If there're subqueries do not belong to any query stages of the main query, the main query could get final physical plan and update UI before those subqueries finished. As a result, the UI can not reflect the change from the subqueries, e.g. new nodes generated from subqueries.

Before:

<img width="335" alt="before_aqe_ui" src="https://user-images.githubusercontent.com/16397174/81149758-671a9480-8fb1-11ea-84c4-9a4520e2b08e.png">

After:
<img width="546" alt="after_aqe_ui" src="https://user-images.githubusercontent.com/16397174/81149752-63870d80-8fb1-11ea-9852-f41e11afe216.png">

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

No(AQE feature hasn't been released).

### How was this patch tested?

Tested manually.

Closes #28460 from Ngone51/fix_aqe_ui.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-06 12:52:53 +00:00
Liang-Chi Hsieh 4952f1a03c [SPARK-31365][SQL] Enable nested predicate pushdown per data sources
### What changes were proposed in this pull request?

This patch proposes to replace `NESTED_PREDICATE_PUSHDOWN_ENABLED` with `NESTED_PREDICATE_PUSHDOWN_V1_SOURCE_LIST` which can configure which v1 data sources are enabled with nested predicate pushdown.

### Why are the changes needed?

We added nested predicate pushdown feature that is configured by `NESTED_PREDICATE_PUSHDOWN_ENABLED`. However, this config is all or nothing config, and applies on all data sources.

In order to not introduce API breaking change after enabling nested predicate pushdown, we'd like to set nested predicate pushdown per data sources. Please also refer to the comments https://github.com/apache/spark/pull/27728#discussion_r410829720.

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

No

### How was this patch tested?

Added/Modified unit tests.

Closes #28366 from viirya/SPARK-31365.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-06 04:50:06 +00:00
sychen 588966d696 [SPARK-31590][SQL] Metadata-only queries should not include subquery in partition filters
### What changes were proposed in this pull request?
Metadata-only queries should not include subquery in partition filters.

### Why are the changes needed?

Apply the `OptimizeMetadataOnlyQuery` rule again, will get the exception `Cannot evaluate expression: scalar-subquery`.

### Does this PR introduce any user-facing change?
Yes. When `spark.sql.optimizer.metadataOnly` is enabled, it succeeds when the queries include subquery in partition filters.

### How was this patch tested?

add UT

Closes #28383 from cxzl25/fix_SPARK-31590.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-06 10:56:19 +09:00
Max Gekk bd26429931 [SPARK-31641][SQL] Fix days conversions by JSON legacy parser
### What changes were proposed in this pull request?
Perform days rebasing while converting days from JSON string field. In Spark 2.4 and earlier versions, the days are interpreted as days since the epoch in the hybrid calendar (Julian + Gregorian since 1582-10-15). Since Spark 3.0, the base calendar was switched to Proleptic Gregorian calendar, so, the days should be rebased to represent the same local date.

### Why are the changes needed?
The changes fix a bug and restore compatibility with Spark 2.4 in which:
```scala
scala> spark.read.schema("d date").json(Seq("{'d': '-141704'}").toDS).show
+----------+
|         d|
+----------+
|1582-01-01|
+----------+
```

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

Before:
```scala
scala> spark.read.schema("d date").json(Seq("{'d': '-141704'}").toDS).show
+----------+
|         d|
+----------+
|1582-01-11|
+----------+
```

After:
```scala
scala> spark.read.schema("d date").json(Seq("{'d': '-141704'}").toDS).show
+----------+
|         d|
+----------+
|1582-01-01|
+----------+
```

### How was this patch tested?
Add a test to `JsonSuite`.

Closes #28453 from MaxGekk/json-rebase-legacy-days.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-05 14:15:31 +00:00
Max Gekk bef5828e12 [SPARK-31630][SQL] Fix perf regression by skipping timestamps rebasing after some threshold
### What changes were proposed in this pull request?
Skip timestamps rebasing after a global threshold when there is no difference between Julian and Gregorian calendars. This allows to avoid checking hash maps of switch points, and fixes perf regressions in `toJavaTimestamp()` and `fromJavaTimestamp()`.

### Why are the changes needed?
The changes fix perf regressions of conversions to/from external type `java.sql.Timestamp`.

Before (see the PR's results https://github.com/apache/spark/pull/28440):
```
================================================================================================
Conversion from/to external types
================================================================================================

OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws
Intel(R) Xeon(R) CPU E5-2670 v2  2.50GHz
To/from Java's date-time:                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Timestamp                             376            388          10         13.3          75.2       1.1X
Collect java.sql.Timestamp                         1878           1937          64          2.7         375.6       0.2X
```

After:
```
================================================================================================
Conversion from/to external types
================================================================================================

OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws
Intel(R) Xeon(R) CPU E5-2670 v2  2.50GHz
To/from Java's date-time:                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Timestamp                             249            264          24         20.1          49.8       1.7X
Collect java.sql.Timestamp                         1503           1523          24          3.3         300.5       0.3X
```

Perf improvements in average of:

1. From java.sql.Timestamp is ~ 34%
2. To java.sql.Timestamps is ~16%

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

### How was this patch tested?
By existing test suites `DateTimeUtilsSuite` and `RebaseDateTimeSuite`.

Closes #28441 from MaxGekk/opt-rebase-common-threshold.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-05 14:11:53 +00:00
turbofei 8d1f7d2a4a [SPARK-31467][SQL][TEST] Refactor the sql tests to prevent TableAlreadyExistsException
### What changes were proposed in this pull request?
If we add UT in hive/SQLQuerySuite or other sql test suites and use table named `test`.
We may meet TableAlreadyExistsException.

```
org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException: Table or view 'test' already exists in database 'default'
```

The reason is that, there is some tests that does not clean up the tables/views.
In this PR, I add `withTempViews` for these tests.

### Why are the changes needed?
To fix the TableAlreadyExistsException issue when adding an UT, which uses table named `test` or others, in some sql test suites, such as hive/SQLQuerySuite.

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

### How was this patch tested?

Existed UT.

Closes #28239 from turboFei/SPARK-31467.

Authored-by: turbofei <fwang12@ebay.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-05-05 15:14:33 +09:00
Max Gekk 735771e7b4 [SPARK-31623][SQL][TESTS] Benchmark rebasing of INT96 and TIMESTAMP_MILLIS timestamps in read/write
### What changes were proposed in this pull request?
Add new benchmarks to `DateTimeRebaseBenchmark` for reading/writing timestamps of INT96 and TIMESTAMP_MICROS column types. Here are benchmark results for reading timestamps after 1582 year with default settings (rebasing is off for TIMESTAMP_MICROS/TIMESTAMP_MILLIS,  and rebasing on for INT96):

timestamp type | vectorized off (ns/row) | vectorized on (ns/row)
--|--|--
TIMESTAMP_MICROS| 160.1 | 50.2
INT96 | 215.6 | 117.8
TIMESTAMP_MILLIS | 159.9 | 60.6

### Why are the changes needed?
To compare default timestamp type `TIMESTAMP_MICROS` with other types in the case if an user decides to switch on them.

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

### How was this patch tested?
By running the benchmarks via:
```
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 64-Bit Server VM 1.8.0_252-8u252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

Closes #28431 from MaxGekk/parquet-timestamps-DateTimeRebaseBenchmark.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-05 05:40:15 +00:00
beliefer b9494206a5 [SPARK-31372][SQL][TEST][FOLLOW-UP] Improve ExpressionsSchemaSuite so that easy to track the diff
### What changes were proposed in this pull request?
This PR follows up https://github.com/apache/spark/pull/28194.
As discussed at https://github.com/apache/spark/pull/28194/files#r418418796.
This PR will improve `ExpressionsSchemaSuite` so that easy to track the diff.
Although `ExpressionsSchemaSuite` at line
b7cde42b04/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala (L165)
just want to compare the total size between expected output size and the newest output size, the scalatest framework will output the extra information contains all the content of expected output and newest output.
This PR will try to avoid this issue.
After this PR, the exception looks like below:
```
[info] - Check schemas for expression examples *** FAILED *** (7 seconds, 336 milliseconds)
[info]   340 did not equal 341 Expected 332 blocks in result file but got 333. Try regenerate the result files. (ExpressionsSchemaSuite.scala:167)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:530)
[info]   at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:529)
[info]   at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
[info]   at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:503)
[info]   at org.apache.spark.sql.ExpressionsSchemaSuite.$anonfun$new$1(ExpressionsSchemaSuite.scala:167)
```

### Why are the changes needed?
Make the exception more concise and clear.

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

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

Closes #28430 from beliefer/improve-expressions-schema-suite.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-05 10:04:16 +09:00
Max Gekk 372ccba063
[SPARK-31639] Revert SPARK-27528 Use Parquet logical type TIMESTAMP_MICROS by default
### What changes were proposed in this pull request?
This reverts commit 43a73e387c. It sets `INT96` as the timestamp type while saving timestamps to parquet files.

### Why are the changes needed?
To be compatible with Hive and Presto that don't support the `TIMESTAMP_MICROS` type in current stable releases.

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

### How was this patch tested?
By existing test suites.

Closes #28450 from MaxGekk/parquet-int96.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-04 17:27:02 -07:00
Burak Yavuz 02a319d7e1 [SPARK-31624] Fix SHOW TBLPROPERTIES for V2 tables that leverage the session catalog
## What changes were proposed in this pull request?

SHOW TBLPROPERTIES does not get the correct table properties for tables using the Session Catalog. This PR fixes that, by explicitly falling back to the V1 implementation if the table is in fact a V1 table. We also hide the reserved table properties for V2 tables, as users do not have control over setting these table properties. Henceforth, if they cannot be set or controlled by the user, then they shouldn't be displayed as such.

### Why are the changes needed?

Shows the incorrect table properties, i.e. only what exists in the Hive MetaStore for V2 tables that may have table properties outside of the MetaStore.

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

Fixes a bug

### How was this patch tested?

Regression test

Closes #28434 from brkyvz/ddlCommands.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-04 12:22:29 +00:00
Wenchen Fan f72220b8ab [SPARK-31606][SQL] Reduce the perf regression of vectorized parquet reader caused by datetime rebase
### What changes were proposed in this pull request?

Push the rebase logic to the lower level of the parquet vectorized reader, to make the final code more vectorization-friendly.

### Why are the changes needed?

Parquet vectorized reader is carefully implemented, to make it more likely to be vectorized by the JVM. However, the newly added datetime rebase degrade the performance a lot, as it breaks vectorization, even if the datetime values don't need to rebase (this is very likely as dates before 1582 is rare).

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

no

### How was this patch tested?

Run part of the `DateTimeRebaseBenchmark` locally. The results:
before this patch
```
[info] Load dates from parquet:                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] after 1582, vec on, rebase off                     2677           2838         142         37.4          26.8       1.0X
[info] after 1582, vec on, rebase on                      3828           4331         805         26.1          38.3       0.7X
[info] before 1582, vec on, rebase off                    2903           2926          34         34.4          29.0       0.9X
[info] before 1582, vec on, rebase on                     4163           4197          38         24.0          41.6       0.6X

[info] Load timestamps from parquet:             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] after 1900, vec on, rebase off                     3537           3627         104         28.3          35.4       1.0X
[info] after 1900, vec on, rebase on                      6891           7010         105         14.5          68.9       0.5X
[info] before 1900, vec on, rebase off                    3692           3770          72         27.1          36.9       1.0X
[info] before 1900, vec on, rebase on                     7588           7610          30         13.2          75.9       0.5X
```

After this patch
```
[info] Load dates from parquet:                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] after 1582, vec on, rebase off                     2758           2944         197         36.3          27.6       1.0X
[info] after 1582, vec on, rebase on                      2908           2966          51         34.4          29.1       0.9X
[info] before 1582, vec on, rebase off                    2840           2878          37         35.2          28.4       1.0X
[info] before 1582, vec on, rebase on                     3407           3433          24         29.4          34.1       0.8X

[info] Load timestamps from parquet:             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] after 1900, vec on, rebase off                     3861           4003         139         25.9          38.6       1.0X
[info] after 1900, vec on, rebase on                      4194           4283          77         23.8          41.9       0.9X
[info] before 1900, vec on, rebase off                    3849           3937          79         26.0          38.5       1.0X
[info] before 1900, vec on, rebase on                     7512           7546          55         13.3          75.1       0.5X
```

Date type is 30% faster if the values don't need to rebase, 20% faster if need to rebase.
Timestamp type is 60% faster if the values don't need to rebase, no difference if need to rebase.

Closes #28406 from cloud-fan/perf.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-04 15:30:10 +09:00
Tianshi Zhu a222644e1d [SPARK-31267][SQL] Flaky test: WholeStageCodegenSparkSubmitSuite.Generated code on driver should not embed platform-specific constant
### What changes were proposed in this pull request?

Allow customized timeouts for `runSparkSubmit`, which will make flaky tests more likely to pass by using a larger timeout value.

I was able to reproduce the test failure on my laptop, which took 1.5 - 2 minutes to finish the test. After increasing the timeout, the test now can pass locally.

### Why are the changes needed?

This allows slow tests to use a larger timeout, so they are more likely to succeed.

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

No

### How was this patch tested?

The test was able to pass on my local env after the change.

Closes #28438 from tianshizz/SPARK-31267.

Authored-by: Tianshi Zhu <zhutianshirea@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-04 14:50:38 +09:00
Max Gekk 2fb85f6b68 [SPARK-31527][SQL][TESTS][FOLLOWUP] Fix the number of rows in DateTimeBenchmark
### What changes were proposed in this pull request?
- Changed to the number of rows in benchmark cases from 3 to the actual number `N`.
- Regenerated benchmark results 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 |

### Why are the changes needed?
The changes are needed to have:
- Correct benchmark results
- Base line for other perf improvements that can be checked in the same environment.

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

### How was this patch tested?
By running the benchmark and checking its output.

Closes #28440 from MaxGekk/SPARK-31527-DateTimeBenchmark-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-05-04 09:39:50 +09:00
Max Gekk 13dddee9a8 [MINOR][SQL][TESTS] Disable UI in SQL benchmarks by default
### What changes were proposed in this pull request?
Set `spark.ui.enabled` to `false` in `SqlBasedBenchmark.getSparkSession`. This disables UI in all SQL benchmarks by default.

### Why are the changes needed?
UI overhead lowers numbers in the `Relative` column and impacts on `Stdev` in benchmark results.

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

### How was this patch tested?
Checked by running `DateTimeRebaseBenchmark`.

Closes #28432 from MaxGekk/ui-off-in-benchmarks.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-05-02 17:54:36 +09:00
Pablo Langa 4fecc20f6e [SPARK-31500][SQL] collect_set() of BinaryType returns duplicate elements
### What changes were proposed in this pull request?

The collect_set() aggregate function should produce a set of distinct elements. When the column argument's type is BinayType this is not the case.

Example:
```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.expressions.Window

case class R(id: String, value: String, bytes: Array[Byte])
def makeR(id: String, value: String) = R(id, value, value.getBytes)
val df = Seq(makeR("a", "dog"), makeR("a", "cat"), makeR("a", "cat"), makeR("b", "fish")).toDF()
// In the example below "bytesSet" erroneously has duplicates but "stringSet" does not (as expected).
df.agg(collect_set('value) as "stringSet", collect_set('bytes) as "byteSet").show(truncate=false)
// The same problem is displayed when using window functions.
val win = Window.partitionBy('id).rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)
val result = df.select(
  collect_set('value).over(win) as "stringSet",
  collect_set('bytes).over(win) as "bytesSet"
)
.select('stringSet, 'bytesSet, size('stringSet) as "stringSetSize", size('bytesSet) as "bytesSetSize")
.show()
```

We use a HashSet buffer to accumulate the results, the problem is that arrays equality in Scala don't behave as expected, arrays ara just plain java arrays and the equality don't compare the content of the arrays
Array(1, 2, 3) == Array(1, 2, 3)  => False
The result is that duplicates are not removed in the hashset

The solution proposed is that in the last stage, when we have all the data in the Hashset buffer, we delete duplicates changing the type of the elements and then transform it to the original type.
This transformation is only applied when we have a BinaryType

### Why are the changes needed?
Fix the bug explained

### Does this PR introduce any user-facing change?
Yes. Now `collect_set()` correctly deduplicates array of byte.

### How was this patch tested?
Unit testing

Closes #28351 from planga82/feature/SPARK-31500_COLLECT_SET_bug.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-05-01 22:09:04 +09:00
Yuanjian Li aec8b69435 [SPARK-28424][TESTS][FOLLOW-UP] Add test cases for all interval units
### What changes were proposed in this pull request?
Add test cases covering all interval units:  MICROSECOND MILLISECOND SECOND MINUTE HOUR DAY WEEK MONTH YEAR

### Why are the changes needed?
For test coverage.

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

### How was this patch tested?
Test only.

Closes #28418 from xuanyuanking/SPARK-28424.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-01 10:32:37 +09:00
Yuanjian Li 7195a18bf2 [SPARK-27340][SS][TESTS][FOLLOW-UP] Rephrase API comments and simplify tests
### What changes were proposed in this pull request?

- Rephrase the API doc for `Column.as`
- Simplify the UTs

### Why are the changes needed?
Address comments in https://github.com/apache/spark/pull/28326

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

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

Closes #28390 from xuanyuanking/SPARK-27340-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-30 06:24:00 +00:00
beliefer 1d1bb79bc6 [SPARK-31372][SQL][TEST] Display expression schema for double check
### What changes were proposed in this pull request?
Although SPARK-30184 Implement a helper method for aliasing functions, developers always forget to using this improvement.
We need to add more powerful guarantees so that aliases outputed by built-in functions are correct.
This PR extracts the SQL from the example of expressions, and output the SQL and its schema into one golden file.
By checking the golden file, we can find the expressions whose aliases are not displayed correctly, and then fix them.

### Why are the changes needed?
Ensure that the output alias is correct

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

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

Closes #28194 from beliefer/check-expression-schema.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-30 03:58:04 +00:00
Kent Yao 9241f8282f [SPARK-31586][SQL][FOLLOWUP] Restore SQL string for datetime - interval operations
### What changes were proposed in this pull request?

Because of  ebc8fa50d0 and  beec8d535f, the SQL output strings for date/timestamp - interval operation will have a malformed format, such as `struct<dateval:date,dateval + (- INTERVAL '2 years 2 months').....`

This PR restore this behavior by adding one `RuntimeReplaceable `implementation for both of the operations to have their pretty SQL strings back.

### Why are the changes needed?

restore the SQL string for datetime operations

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

NO, we are restoring here
### How was this patch tested?

added unit tests

Closes #28402 from yaooqinn/SPARK-31586-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-30 03:31:29 +00:00
Max Gekk 91648654da [SPARK-31553][SQL][TESTS][FOLLOWUP] Tests for collection elem types of isInCollection
### What changes were proposed in this pull request?
- Add tests for different element types of collections that could be passed to `isInCollection`. Added tests for types that can pass the check `In`.`checkInputDataTypes()`.
- Test different switch thresholds in the `isInCollection: Scala Collection` test.

### Why are the changes needed?
To prevent regressions like introduced by https://github.com/apache/spark/pull/25754 and reverted by https://github.com/apache/spark/pull/28388

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

### How was this patch tested?
By existing and new tests in `ColumnExpressionSuite`

Closes #28405 from MaxGekk/test-isInCollection.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-30 03:20:10 +00:00
Takeshi Yamamuro 97f2c03d3b
[SPARK-31594][SQL] Do not display the seed of rand/randn with no argument in output schema
### What changes were proposed in this pull request?

This PR intends to update `sql` in `Rand`/`Randn` with no argument to make a column name deterministic.

Before this PR (a column name changes run-by-run):
```
scala> sql("select rand()").show()
+-------------------------+
|rand(7986133828002692830)|
+-------------------------+
|       0.9524061403696937|
+-------------------------+
```
After this PR (a column name fixed):
```
scala> sql("select rand()").show()
+------------------+
|            rand()|
+------------------+
|0.7137935639522275|
+------------------+

// If a seed given, it is still shown in a column name
// (the same with the current behaviour)
scala> sql("select rand(1)").show()
+------------------+
|           rand(1)|
+------------------+
|0.6363787615254752|
+------------------+

// We can still check a seed in explain output:
scala> sql("select rand()").explain()
== Physical Plan ==
*(1) Project [rand(-2282124938778456838) AS rand()#0]
+- *(1) Scan OneRowRelation[]
```

Note: This fix comes from #28194; the ongoing PR tests the output schema of expressions, so their schemas must be deterministic for the tests.

### Why are the changes needed?

To make output schema deterministic.

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

No.

### How was this patch tested?

Added unit tests.

Closes #28392 from maropu/SPARK-31594.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-29 00:14:50 -07:00
Terry Kim 36803031e8 [SPARK-30282][SQL][FOLLOWUP] SHOW TBLPROPERTIES should support views
### What changes were proposed in this pull request?

This PR addresses two things:
- `SHOW TBLPROPERTIES` should supports view (a regression introduced by #26921)
- `SHOW TBLPROPERTIES` on a temporary view should return empty result (2.4 behavior instead of throwing `AnalysisException`.

### Why are the changes needed?

It's a bug.

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

Yes, now `SHOW TBLPROPERTIES` works on views:
```
scala> sql("CREATE VIEW view TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1")
scala> sql("SHOW TBLPROPERTIES view").show(truncate=false)
+---------------------------------+-------------+
|key                              |value        |
+---------------------------------+-------------+
|view.catalogAndNamespace.numParts|2            |
|view.query.out.col.0             |c1           |
|view.query.out.numCols           |1            |
|p2                               |v2           |
|view.catalogAndNamespace.part.0  |spark_catalog|
|p1                               |v1           |
|view.catalogAndNamespace.part.1  |default      |
+---------------------------------+-------------+
```
And for a temporary view:
```
scala> sql("CREATE TEMPORARY VIEW tview TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1")
scala> sql("SHOW TBLPROPERTIES tview").show(truncate=false)
+---+-----+
|key|value|
+---+-----+
+---+-----+
```

### How was this patch tested?

Added tests.

Closes #28375 from imback82/show_tblproperties_followup.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-29 07:06:45 +00:00
Kent Yao ea525fe8c0 [SPARK-31597][SQL] extracting day from intervals should be interval.days + days in interval.microsecond
### What changes were proposed in this pull request?

With suggestion from cloud-fan https://github.com/apache/spark/pull/28222#issuecomment-620586933

I Checked with both Presto and PostgresSQL, one is implemented intervals with ANSI style year-month/day-time, and the other is mixed and Non-ANSI. They both add the exceeded days in interval time part to the total days of the operation which extracts day from interval values.

```sql

presto> SELECT EXTRACT(DAY FROM (cast('2020-01-15 00:00:00' as timestamp) - cast('2020-01-01 00:00:00' as timestamp)));
_col0
-------
14
(1 row)

Query 20200428_135239_00000_ahn7x, FINISHED, 1 node
Splits: 17 total, 17 done (100.00%)
0:01 [0 rows, 0B] [0 rows/s, 0B/s]

presto> SELECT EXTRACT(DAY FROM (cast('2020-01-15 00:00:00' as timestamp) - cast('2020-01-01 00:00:01' as timestamp)));
_col0
-------
13
(1 row)

Query 20200428_135246_00001_ahn7x, FINISHED, 1 node
Splits: 17 total, 17 done (100.00%)
0:00 [0 rows, 0B] [0 rows/s, 0B/s]

presto>

```

```sql

postgres=# SELECT EXTRACT(DAY FROM (cast('2020-01-15 00:00:00' as timestamp) - cast('2020-01-01 00:00:00' as timestamp)));
date_part
-----------
14
(1 row)

postgres=# SELECT EXTRACT(DAY FROM (cast('2020-01-15 00:00:00' as timestamp) - cast('2020-01-01 00:00:01' as timestamp)));
date_part
-----------
13

```

```
spark-sql> SELECT EXTRACT(DAY FROM (cast('2020-01-15 00:00:00' as timestamp) - cast('2020-01-01 00:00:01' as timestamp)));
0
spark-sql> SELECT EXTRACT(DAY FROM (cast('2020-01-15 00:00:00' as timestamp) - cast('2020-01-01 00:00:00' as timestamp)));
0
```

In ANSI standard, the day is exact 24 hours, so we don't need to worry about the conceptual day for interval extraction. The meaning of the conceptual day only takes effect when we add it to a zoned timestamp value.

### Why are the changes needed?

Both satisfy the ANSI standard and common use cases in modern SQL platforms

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

No, it new in 3.0
### How was this patch tested?

add more uts

Closes #28396 from yaooqinn/SPARK-31597.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-29 06:56:33 +00:00
Kent Yao 295d866969 [SPARK-31596][SQL][DOCS] Generate SQL Configurations from hive module to configuration doc
### What changes were proposed in this pull request?

This PR adds `-Phive` profile to the pre-build phase to build the hive module to dev classpath.
Then reflect the HiveUtils object to dump all configurations in the class.

### Why are the changes needed?

supply SQL configurations from hive module to doc

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

NO

### How was this patch tested?

passing Jenkins
 add verified locally

![image](https://user-images.githubusercontent.com/8326978/80492333-6fae1200-8996-11ea-99fd-595ee18c67e5.png)

Closes #28394 from yaooqinn/SPARK-31596.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-04-29 15:34:45 +09:00
Kent Yao 54996be4d2 [SPARK-31527][SQL][TESTS][FOLLOWUP] Add a benchmark test for datetime add/subtract interval operations
### What changes were proposed in this pull request?
With https://github.com/apache/spark/pull/28310, the operation of date +/- interval(m, d, 0) has been improved a lot.

According to the benchmark results, about 75% time cost is reduced because of no casting date to timestamp back and forth.

In this PR, we add a benchmark for these operations, and timestamp +/- interval operations as accessories.

### Why are the changes needed?

Performance test coverage, since these operations are missing in the DateTimeBenchmark.

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

No, just test

### How was this patch tested?

regenerated benchmark results

Closes #28369 from yaooqinn/SPARK-31527-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-28 15:39:28 +00:00
Max Gekk b7cabc80e6 [SPARK-31553][SQL] Revert "[SPARK-29048] Improve performance on Column.isInCollection() with a large size collection"
### What changes were proposed in this pull request?
This reverts commit 5631a96367.

Closes #28328

### Why are the changes needed?
The PR  https://github.com/apache/spark/pull/25754 introduced a bug in `isInCollection`. For example, if the SQL config `spark.sql.optimizer.inSetConversionThreshold`is set to 10 (by default):
```scala
val set = (0 to 20).map(_.toString).toSet
val data = Seq("1").toDF("x")
data.select($"x".isInCollection(set).as("isInCollection")).show()
```
The function must return **'true'** because "1" is in the set of "0" ... "20" but it returns "false":
```
+--------------+
|isInCollection|
+--------------+
|         false|
+--------------+
```

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

### How was this patch tested?
```
$ ./build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #28388 from MaxGekk/fix-isInCollection-revert.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-28 14:10:50 +00:00
Kent Yao beec8d535f [SPARK-31586][SQL] Replace expression TimeSub(l, r) with TimeAdd(l -r)
### What changes were proposed in this pull request?

The implementation of TimeSub for the operation of timestamp subtracting interval is almost repetitive with TimeAdd. We can replace it with TimeAdd(l, -r) since there are equivalent.

Suggestion from https://github.com/apache/spark/pull/28310#discussion_r414259239

Besides, the Coercion rules for TimeAdd/TimeSub(date, interval) are useless anymore, so remove them in this PR since they are touched in this PR.

### Why are the changes needed?

remove redundant and useless code for easy maintenance

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

Yes, the SQL string of `datetime - interval` become `datetime + (- interval)`
### How was this patch tested?

modified existing unit tests.

Closes #28381 from yaooqinn/SPARK-31586.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-28 14:01:07 +00:00
Yuanjian Li 6ed2dfbba1 [SPARK-31519][SQL] Cast in having aggregate expressions returns the wrong result
### What changes were proposed in this pull request?
Add a new logical node AggregateWithHaving, and the parser should create this plan for HAVING. The analyzer resolves it to Filter(..., Aggregate(...)).

### Why are the changes needed?
The SQL parser in Spark creates Filter(..., Aggregate(...)) for the HAVING query, and Spark has a special analyzer rule ResolveAggregateFunctions to resolve the aggregate functions and grouping columns in the Filter operator.

It works for simple cases in a very tricky way as it relies on rule execution order:
1. Rule ResolveReferences hits the Aggregate operator and resolves attributes inside aggregate functions, but the function itself is still unresolved as it's an UnresolvedFunction. This stops resolving the Filter operator as the child Aggrege operator is still unresolved.
2. Rule ResolveFunctions resolves UnresolvedFunction. This makes the Aggrege operator resolved.
3. Rule ResolveAggregateFunctions resolves the Filter operator if its child is a resolved Aggregate. This rule can correctly resolve the grouping columns.

In the example query, I put a CAST, which needs to be resolved by rule ResolveTimeZone, which runs after ResolveAggregateFunctions. This breaks step 3 as the Aggregate operator is unresolved at that time. Then the analyzer starts next round and the Filter operator is resolved by ResolveReferences, which wrongly resolves the grouping columns.

See the demo below:
```
SELECT SUM(a) AS b, '2020-01-01' AS fake FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY b HAVING b > 10
```
The query's result is
```
+---+----------+
|  b|      fake|
+---+----------+
|  2|2020-01-01|
+---+----------+
```
But if we add CAST, it will return an empty result.
```
SELECT SUM(a) AS b, CAST('2020-01-01' AS DATE) AS fake FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY b HAVING b > 10
```

### Does this PR introduce any user-facing change?
Yes, bug fix for cast in having aggregate expressions.

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

Closes #28294 from xuanyuanking/SPARK-31519.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-28 08:11:41 +00:00
jiake 079b3623c8 [SPARK-31524][SQL] Add metric to the split task number for skew optimization
### What changes were proposed in this pull request?
This is a followup of [#28022](https://github.com/apache/spark/pull/28022), to add the metric info of split task number for skewed optimization.
With this PR, we can see the number of splits for the skewed partitions as following:
![image](https://user-images.githubusercontent.com/11972570/80294583-ff886c00-879c-11ea-813c-2db302f99f04.png)

### Why are the changes needed?
make UI more friendly

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

### How was this patch tested?
existing ut

Closes #28109 from JkSelf/addSplitNumer.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-28 07:21:00 +00:00
Dongjoon Hyun 79eaaaf6da
[SPARK-31580][BUILD] Upgrade Apache ORC to 1.5.10
### What changes were proposed in this pull request?

This PR aims to upgrade Apache ORC to 1.5.10.

### Why are the changes needed?

Apache ORC 1.5.10 is a maintenance release with the following patches.

- [ORC-621](https://issues.apache.org/jira/browse/ORC-621) Need reader fix for ORC-569
- [ORC-616](https://issues.apache.org/jira/browse/ORC-616) In Patched Base encoding, the value of headerThirdByte goes beyond the range of byte
- [ORC-613](https://issues.apache.org/jira/browse/ORC-613) OrcMapredRecordReader mis-reuse struct object when actual children schema differs
- [ORC-610](https://issues.apache.org/jira/browse/ORC-610) Updated Copyright year in the NOTICE file

The following is release note.
- https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12318320&version=12346912

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

No.

### How was this patch tested?

Pass the Jenkins with the existing ORC tests and a newly added test case.

- The first commit is already tested in `hive-2.3` profile with both native ORC implementation and Hive 2.3 ORC implementation. (https://github.com/apache/spark/pull/28373#issuecomment-620265114)
- The latest run is about to make the test case disable in `hive-1.2` profile which doesn't use Apache ORC.
  - `hive-1.2`: https://github.com/apache/spark/pull/28373#issuecomment-620325906

Closes #28373 from dongjoon-hyun/SPARK-ORC-1.5.10.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-27 18:56:30 -07:00
Wenchen Fan 2f4f38b6f1
[SPARK-31577][SQL] Fix case-sensitivity and forward name conflict problems when check name conflicts of CTE relations
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/28318, to make the code more readable, by adding some comments to explain the trick and simplify the code to use a boolean flag instead of 2 string sets.

This PR also fixes various problems:
1. the name check should consider case sensitivity
2. forward name conflicts like `with t as (with t2 as ...), t2 as ...` is not a real conflict and we shouldn't fail.

### Why are the changes needed?

correct the behavior

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

yes, fix the fore-mentioned behaviors.

### How was this patch tested?

new tests

Closes #28371 from cloud-fan/followup.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-27 16:47:39 -07:00
Yuanjian Li ba7adc4949
[SPARK-27340][SS] Alias on TimeWindow expression cause watermark metadata lost
Credit to LiangchangZ, this PR reuses the UT as well as integrate test in #24457. Thanks Liangchang for your solid work.

### What changes were proposed in this pull request?
Make metadata propagatable between Aliases.

### Why are the changes needed?
In Structured Streaming, we added an Alias for TimeWindow by default.
590b9a0132/sql/core/src/main/scala/org/apache/spark/sql/functions.scala (L3272-L3273)
For some cases like stream join with watermark and window, users need to add an alias for convenience(we also added one in StreamingJoinSuite). The current metadata handling logic for `as` will lose the watermark metadata
590b9a0132/sql/core/src/main/scala/org/apache/spark/sql/Column.scala (L1049-L1054)
 and finally cause the AnalysisException:
```
Stream-stream outer join between two streaming DataFrame/Datasets is not supported without a watermark in the join keys, or a watermark on the nullable side and an appropriate range condition
```

### Does this PR introduce any user-facing change?
Bugfix for an alias on time window with watermark.

### How was this patch tested?
New UTs added. One for the functionality and one for explaining the common scenario.

Closes #28326 from xuanyuanking/SPARK-27340.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-27 15:07:52 -07:00
Kousuke Saruta 7d4d05c684
[SPARK-31565][WEBUI][FOLLOWUP] Add font color setting of DAG-viz for query plan
### What changes were proposed in this pull request?

This PR adds a font color setting of DAG-viz for query plan.

### Why are the changes needed?

#28352 aimed to unify the font color of all DAG-viz in WebUI but there is one part left over.

Before this change applied, the appearance of a query plan is like as follows.
<img width="456" alt="plan-graph-fixed" src="https://user-images.githubusercontent.com/4736016/80325600-ca4d4e00-8870-11ea-945c-64971dbb752c.png">
The color of `WholeStageCodegen (1)` and its following text (`duration: total...`) is slightly darker than `SerializeFromObject`.

After this change, those color is unified as `#333333`.
<img width="450" alt="plan-graph-fixed2" src="https://user-images.githubusercontent.com/4736016/80325651-fb2d8300-8870-11ea-8ed8-178c124d224c.png">

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

Slightly yes.

### How was this patch tested?

I confirmed the style of `fill` and `color` is `#333333` by debug console of Chrome.
<img width="321" alt="fill" src="https://user-images.githubusercontent.com/4736016/80325760-6c6d3600-8871-11ea-82e7-e789bf741f2a.png">
<img width="316" alt="color" src="https://user-images.githubusercontent.com/4736016/80325765-70995380-8871-11ea-8976-7020205d585c.png">

Closes #28355 from sarutak/followup-SPARK-31565.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-27 13:34:43 -07:00
Kent Yao 5ba467ca1d [SPARK-31550][SQL][DOCS] Set nondeterministic configurations with general meanings in sql configuration doc
### What changes were proposed in this pull request?

```scala
spark.sql.session.timeZone

spark.sql.warehouse.dir
```
these 2 configs are nondeterministic and vary with environments

Besides, reflect code in `gen-sql-config-docs.py` via  https://github.com/apache/spark/pull/28274#discussion_r412893096 and `configuration.md` via https://github.com/apache/spark/pull/28274#discussion_r412894905
### Why are the changes needed?

doc fix

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

no
### How was this patch tested?

verify locally
![image](https://user-images.githubusercontent.com/8326978/80179099-5e7da200-8632-11ea-803f-d47a93151869.png)

Closes #28322 from yaooqinn/SPARK-31550.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-27 17:08:52 +09:00
yi.wu 7df658414b [SPARK-31529][SQL] Remove extra whitespaces in formatted explain
### What changes were proposed in this pull request?

Remove all the extra whitespaces in the formatted explain.

### Why are the changes needed?

The number of extra whitespaces of the formatted explain becomes different between master and branch-3.0. This causes a problem that whenever we backport formatted explain related tests from master to branch-3.0, it will fail branch-3.0. Besides, extra whitespaces are always disallowed in Spark. Thus, we should remove them as possible as we can.

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

No, formatted explain is newly added in Spark 3.0.

### How was this patch tested?

Updated sql query tests.

Closes #28315 from Ngone51/fix_extra_spaces.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-27 07:39:24 +00:00
Kent Yao ebc8fa50d0 [SPARK-31527][SQL] date add/subtract interval only allow those day precision in ansi mode
### What changes were proposed in this pull request?

To follow ANSI,the expressions - `date + interval`, `interval + date` and `date - interval` should only accept intervals which the `microseconds` part is 0.

### Why are the changes needed?

Better ANSI compliance

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

No, this PR should target 3.0.0 in which this feature is newly added.

### How was this patch tested?

add more unit tests

Closes #28310 from yaooqinn/SPARK-31527.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-27 05:28:46 +00:00
Kousuke Saruta 91ec2eacfa
[SPARK-31565][WEBUI] Unify the font color of label among all DAG-viz
### What changes were proposed in this pull request?

This PR unifies the font color of label as `#333333` among all DAG-viz.

### Why are the changes needed?

For the consistent appearance among all DAG-viz.
There are three types of DAG-viz in the WebUI.
One is for stages, another one is for RDDs and the last one is for query plans.
But the font color of labels are slightly different among them.

For stages, the color is `#333333` (simply 333) which is specified by `spark-dag-viz.css`.
<img width="355" alt="job-graph" src="https://user-images.githubusercontent.com/4736016/80321397-b517f580-8857-11ea-8c8e-cf68f648ab05.png">
<img width="310" alt="job-graph-color" src="https://user-images.githubusercontent.com/4736016/80321399-ba754000-8857-11ea-8708-83bdef4bc1d1.png">

For RDDs, the color is `#212529` which is specified by `bootstrap.min.js`.
<img width="386" alt="stage-graph" src="https://user-images.githubusercontent.com/4736016/80321438-f0b2bf80-8857-11ea-9c2a-13fa0fd1431c.png">
<img width="313" alt="stage-graph-color" src="https://user-images.githubusercontent.com/4736016/80321444-fa3c2780-8857-11ea-81b2-4f1203d47896.png">

For query plans, the color is `black` which is specified by `spark-sql-viz.css`.
<img width="449" alt="plan-graph" src="https://user-images.githubusercontent.com/4736016/80321490-61f27280-8858-11ea-9c3a-2c98d3d4d03b.png">
<img width="316" alt="plan-graph-color" src="https://user-images.githubusercontent.com/4736016/80321496-6ae34400-8858-11ea-8fe8-0d6e4a821608.png">

After the change, the appearance is like as follows (no change for stages).

For RDDs.
<img width="389" alt="stage-graph-fixed" src="https://user-images.githubusercontent.com/4736016/80321613-6b300f00-8859-11ea-912f-d92474aa9f47.png">

For query plans.
<img width="456" alt="plan-graph-fixed" src="https://user-images.githubusercontent.com/4736016/80321638-9a468080-8859-11ea-974c-33c56a8ffe1a.png">

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

Yes. The unified color is slightly lighter than ever.

### How was this patch tested?

Confirmed that the color code among all DAG-viz are `#333333` using browser's debug console.

Closes #28352 from sarutak/unify-label-color.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-26 16:57:23 -07:00
Max Gekk bd139bda4a
[SPARK-31489][SQL] Fix pushing down filters with java.time.LocalDate values in ORC
### What changes were proposed in this pull request?
Convert `java.time.LocalDate` to `java.sql.Date` in pushed down filters to ORC datasource when Java 8 time API enabled.

Closes #28272

### Why are the changes needed?
The changes fix the exception raised while pushing date filters when `spark.sql.datetime.java8API.enabled` is set to `true`:
```
Wrong value class java.time.LocalDate for DATE.EQUALS leaf
java.lang.IllegalArgumentException: Wrong value class java.time.LocalDate for DATE.EQUALS leaf
	at org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl$PredicateLeafImpl.checkLiteralType(SearchArgumentImpl.java:192)
	at org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl$PredicateLeafImpl.<init>(SearchArgumentImpl.java:75)
	at org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl$BuilderImpl.equals(SearchArgumentImpl.java:352)
	at org.apache.spark.sql.execution.datasources.orc.OrcFilters$.buildLeafSearchArgument(OrcFilters.scala:229)
```

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

### How was this patch tested?
Added tests to `OrcFilterSuite`.

Closes #28261 from MaxGekk/orc-date-filter-pushdown.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-26 15:49:00 -07:00
Peter Toth 4f53bfbbd5
[SPARK-31535][SQL] Fix nested CTE substitution
### What changes were proposed in this pull request?

This PR fixes a CTE substitution issue so as to the following SQL return the correct empty result:
```
WITH t(c) AS (SELECT 1)
SELECT * FROM t
WHERE c IN (
  WITH t(c) AS (SELECT 2)
  SELECT * FROM t
)
```
Before this PR the result was `1`.

### Why are the changes needed?
To fix a correctness issue.

### Does this PR introduce any user-facing change?
Yes, fixes a correctness issue.

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

Closes #28318 from peter-toth/SPARK-31535-fix-nested-cte-substitution.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-26 15:31:32 -07:00
Takeshi Yamamuro e01125db0d
[SPARK-31562][SQL] Update ExpressionDescription for substring, current_date, and current_timestamp
### What changes were proposed in this pull request?

This PR intends to add entries for substring, current_date, and current_timestamp in the SQL built-in function documents. Specifically, the entries are as follows;

 - SELECT current_date;
 - SELECT current_timestamp;
 - SELECT substring('abcd' FROM 1);
 - SELECT substring('abcd' FROM 1 FOR 2);

### Why are the changes needed?

To make the SQL (built-in functions) references complete.

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

<img width="1040" alt="Screen Shot 2020-04-25 at 16 51 07" src="https://user-images.githubusercontent.com/692303/80274851-6ca5ee00-8718-11ea-9a35-9ae82008cb4b.png">

<img width="974" alt="Screen Shot 2020-04-25 at 17 24 24" src="https://user-images.githubusercontent.com/692303/80275032-a88d8300-8719-11ea-92ec-95b80169ae28.png">

<img width="862" alt="Screen Shot 2020-04-25 at 17 27 48" src="https://user-images.githubusercontent.com/692303/80275114-36696e00-871a-11ea-8e39-02e93eabb92f.png">

### How was this patch tested?

Added test examples.

Closes #28342 from maropu/SPARK-31562.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-26 11:46:52 -07:00
Gengliang Wang f59ebdef5b [SPARK-31558][UI] Code clean up in spark-sql-viz.js
### What changes were proposed in this pull request?

1. Remove console.log(), which seems unnecessary in the releases.
2. Replace the double equals to triple equals
3. Reuse jquery selector.

### Why are the changes needed?

For better code quality.

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

No

### How was this patch tested?

Existing tests + manual test.

Closes #28333 from gengliangwang/removeLog.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-04-25 13:43:52 -07:00
Kent Yao 7959808e96
[SPARK-31564][TESTS] Fix flaky AllExecutionsPageSuite for checking 1970
### What changes were proposed in this pull request?

Fix flakiness by checking `1970/01/01` instead of `1970`.
The test was added by SPARK-27125 for 3.0.0.

### Why are the changes needed?

the `org.apache.spark.sql.execution.ui.AllExecutionsPageSuite.SPARK-27019:correctly display SQL page when event reordering happens` test is flaky for just checking the `html` content not containing 1970. I will add a ticket to check and fix that.
In the specific failure https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/121799/testReport, it failed because the `html`

```
...
<td sorttable_customkey="1587806019707">
...

```
contained `1970`.

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

no

### How was this patch tested?

passing jenkins

Closes #28344 from yaooqinn/SPARK-31564.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-25 10:27:05 -07:00
Max Gekk 7d8216a664
[SPARK-31563][SQL] Fix failure of InSet.sql for collections of Catalyst's internal types
### What changes were proposed in this pull request?
In the PR, I propose to fix the `InSet.sql` method for the cases when input collection contains values of internal Catalyst's types, for instance `UTF8String`. Elements of the input set `hset` are converted to Scala types, and wrapped by `Literal` to properly form SQL view of the input collection.

### Why are the changes needed?
The changes fixed the bug in `InSet.sql` that makes wrong assumption about types of collection elements. See more details in SPARK-31563.

### Does this PR introduce any user-facing change?
Highly likely, not.

### How was this patch tested?
Added a test to `ColumnExpressionSuite`

Closes #28343 from MaxGekk/fix-InSet-sql.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-25 09:29:51 -07:00
Gengliang Wang 16b961526d
[SPARK-31560][SQL][TESTS] Add V1/V2 tests for TextSuite and WholeTextFileSuite
### What changes were proposed in this pull request?

 Add V1/V2 tests for TextSuite and WholeTextFileSuite

### Why are the changes needed?

This is missing part since #24207. We should have these tests for test coverage.

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

No

### How was this patch tested?

Unit tests.

Closes #28335 from gengliangwang/testV2Suite.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-24 18:59:15 -07:00
Kent Yao f92652d0b5
[SPARK-31528][SQL] Remove millennium, century, decade from trunc/date_trunc fucntions
### What changes were proposed in this pull request?

Similar to https://jira.apache.org/jira/browse/SPARK-31507, millennium, century, and decade are not commonly used in most modern platforms.

For example
Negative:
https://docs.snowflake.com/en/sql-reference/functions-date-time.html#supported-date-and-time-parts
https://prestodb.io/docs/current/functions/datetime.html#date_trunc
https://teradata.github.io/presto/docs/148t/functions/datetime.html#date_trunc
https://www.oracletutorial.com/oracle-date-functions/oracle-trunc/

Positive:
https://docs.aws.amazon.com/redshift/latest/dg/r_Dateparts_for_datetime_functions.html
https://www.postgresql.org/docs/9.1/functions-datetime.html#FUNCTIONS-DATETIME-TRUNC

This PR removes these `fmt`s support for trunc and date_trunc functions.

### Why are the changes needed?

clean uncommon datetime unit for easy maintenance, we can add them back if they are found very useful later.

### Does this PR introduce any user-facing change?
no, targeting 3.0.0, these are newly added in 3.0.0

### How was this patch tested?

remove and modify existing units tests

Closes #28313 from yaooqinn/SPARK-31528.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-24 18:28:41 -07:00
Kent Yao caf3ab8411
[SPARK-31552][SQL] Fix ClassCastException in ScalaReflection arrayClassFor
### What changes were proposed in this pull request?

the 2 method `arrayClassFor` and `dataTypeFor` in `ScalaReflection` call each other circularly, the cases in `dataTypeFor` are not fully handled in `arrayClassFor`

For example:
```scala
scala> implicit def newArrayEncoder[T <: Array[_] : TypeTag]: Encoder[T] = ExpressionEncoder()
newArrayEncoder: [T <: Array[_]](implicit evidence$1: reflect.runtime.universe.TypeTag[T])org.apache.spark.sql.Encoder[T]

scala> val decOne = Decimal(1, 38, 18)
decOne: org.apache.spark.sql.types.Decimal = 1E-18

scala> val decTwo = Decimal(2, 38, 18)
decTwo: org.apache.spark.sql.types.Decimal = 2E-18

scala> val decSpark = Array(decOne, decTwo)
decSpark: Array[org.apache.spark.sql.types.Decimal] = Array(1E-18, 2E-18)

scala> Seq(decSpark).toDF()
java.lang.ClassCastException: org.apache.spark.sql.types.DecimalType cannot be cast to org.apache.spark.sql.types.ObjectType
  at org.apache.spark.sql.catalyst.ScalaReflection$.$anonfun$arrayClassFor$1(ScalaReflection.scala:131)
  at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:69)
  at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects(ScalaReflection.scala:879)
  at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects$(ScalaReflection.scala:878)
  at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:49)
  at org.apache.spark.sql.catalyst.ScalaReflection$.arrayClassFor(ScalaReflection.scala:120)
  at org.apache.spark.sql.catalyst.ScalaReflection$.$anonfun$dataTypeFor$1(ScalaReflection.scala:105)
  at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:69)
  at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects(ScalaReflection.scala:879)
  at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects$(ScalaReflection.scala:878)
  at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:49)
  at org.apache.spark.sql.catalyst.ScalaReflection$.dataTypeFor(ScalaReflection.scala:88)
  at org.apache.spark.sql.catalyst.ScalaReflection$.$anonfun$serializerForType$1(ScalaReflection.scala:399)
  at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:69)
  at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects(ScalaReflection.scala:879)
  at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects$(ScalaReflection.scala:878)
  at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:49)
  at org.apache.spark.sql.catalyst.ScalaReflection$.serializerForType(ScalaReflection.scala:393)
  at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$.apply(ExpressionEncoder.scala:57)
  at newArrayEncoder(<console>:57)
  ... 53 elided

scala>
```

In this PR, we add the missing cases to `arrayClassFor`

### Why are the changes needed?

bugfix as described above

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

no

### How was this patch tested?

add a test for array encoders

Closes #28324 from yaooqinn/SPARK-31552.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-24 18:04:26 -07:00
Kent Yao 8424f55229 [SPARK-31532][SQL] Builder should not propagate static sql configs to the existing active or default SparkSession
### What changes were proposed in this pull request?

SparkSessionBuilder shoud not propagate static sql configurations to the existing active/default SparkSession
This seems a long-standing bug.

```scala
scala> spark.sql("set spark.sql.warehouse.dir").show
+--------------------+--------------------+
|                 key|               value|
+--------------------+--------------------+
|spark.sql.warehou...|file:/Users/kenty...|
+--------------------+--------------------+

scala> spark.sql("set spark.sql.warehouse.dir=2");
org.apache.spark.sql.AnalysisException: Cannot modify the value of a static config: spark.sql.warehouse.dir;
  at org.apache.spark.sql.RuntimeConfig.requireNonStaticConf(RuntimeConfig.scala:154)
  at org.apache.spark.sql.RuntimeConfig.set(RuntimeConfig.scala:42)
  at org.apache.spark.sql.execution.command.SetCommand.$anonfun$x$7$6(SetCommand.scala:100)
  at org.apache.spark.sql.execution.command.SetCommand.run(SetCommand.scala:156)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
  at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:229)
  at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3644)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
  at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3642)
  at org.apache.spark.sql.Dataset.<init>(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:607)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:602)
  ... 47 elided

scala> import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.SparkSession

scala> SparkSession.builder.config("spark.sql.warehouse.dir", "xyz").get
getClass   getOrCreate

scala> SparkSession.builder.config("spark.sql.warehouse.dir", "xyz").getOrCreate
20/04/23 23:49:13 WARN SparkSession$Builder: Using an existing SparkSession; some configuration may not take effect.
res7: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession6403d574

scala> spark.sql("set spark.sql.warehouse.dir").show
+--------------------+-----+
|                 key|value|
+--------------------+-----+
|spark.sql.warehou...|  xyz|
+--------------------+-----+

scala>
OptionsAttachments
```

### Why are the changes needed?
bugfix as shown in the previous section

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

Yes, static SQL configurations with SparkSession.builder.config do not propagate to any existing or new SparkSession instances.

### How was this patch tested?

new ut.

Closes #28316 from yaooqinn/SPARK-31532.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-04-25 08:53:00 +09:00
Jian Tang 6a576161ae [SPARK-31364][SQL][TESTS] Benchmark Parquet Nested Field Predicate Pushdown
### What changes were proposed in this pull request?

This PR aims to add a benchmark suite for nested predicate pushdown with parquet file:

Performance comparison: Nested predicate pushdown disabled vs enabled,  with the following queries scenarios:

1.  When predicate pushed down, parquet reader are able to filter out all the row groups without loading them.

2. When predicate pushed down, parquet reader only loads one of the row groups.

3. When predicate pushed down, parquet reader can't filter out any row group in order to see if we introduce too much overhead or not when enabling nested predicate push down.

### Why are the changes needed?

No benchmark exists today for nested fields predicate pushdown performance evaluation.

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

### How was this patch tested?
 Benchmark runs and reporting result.

Closes #28319 from JiJiTang/SPARK-31364.

Authored-by: Jian Tang <jian_tang@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-04-24 22:10:58 +00:00
Yuming Wang b10263b8e5 [SPARK-30724][SQL] Support 'LIKE ANY' and 'LIKE ALL' operators
### What changes were proposed in this pull request?

`LIKE ANY/SOME` and `LIKE ALL` operators are mostly used when we are matching a text field with numbers of patterns. For example:

Teradata / Hive 3.0 / Snowflake:
```sql
--like any
select 'foo' LIKE ANY ('%foo%','%bar%');

--like all
select 'foo' LIKE ALL ('%foo%','%bar%');
```
PostgreSQL:
```sql
-- like any
select 'foo' LIKE ANY (array['%foo%','%bar%']);

-- like all
select 'foo' LIKE ALL (array['%foo%','%bar%']);
```

This PR add support these two operators.

More details:
https://docs.teradata.com/reader/756LNiPSFdY~4JcCCcR5Cw/4~AyrPNmDN0Xk4SALLo6aQ
https://issues.apache.org/jira/browse/HIVE-15229
https://docs.snowflake.net/manuals/sql-reference/functions/like_any.html

### Why are the changes needed?

To smoothly migrate SQLs to Spark SQL.

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

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

Closes #27477 from wangyum/SPARK-30724.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-04-24 22:20:32 +09:00
yi.wu 463c54419b [SPARK-31010][SQL][DOC][FOLLOW-UP] Improve deprecated warning message for untyped scala udf
### What changes were proposed in this pull request?

Give more friendly warning message/migration guide of deprecated scala udf to users.

### Why are the changes needed?

User can not distinguish function signature between typed and untyped scala udf. Instead, we shall tell user what to do directly.

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

No, it's newly added in Spark 3.0.

### How was this patch tested?

Pass Jenkins.

Closes #28311 from Ngone51/update_udf_doc.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-24 19:10:18 +09:00
Jungtaek Lim (HeartSaVioR) 39bc50dbf8 [SPARK-30804][SS] Measure and log elapsed time for "compact" operation in CompactibleFileStreamLog
### What changes were proposed in this pull request?

This patch adds some log messages to log elapsed time for "compact" operation in FileStreamSourceLog and FileStreamSinkLog (added in CompactibleFileStreamLog) to help investigating the mysterious latency spike during the batch run.

### Why are the changes needed?

Tracking latency is a critical aspect of streaming query. While "compact" operation may bring nontrivial latency (it's even synchronous, adding all the latency to the batch run), it's not measured and end users have to guess.

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

No.

### How was this patch tested?

N/A for UT. Manual test with streaming query using file source & file sink.

> grep "for compact batch" <driver log>

```
...
20/02/20 19:27:36 WARN FileStreamSinkLog: Compacting took 24473 ms (load: 14185 ms, write: 10288 ms) for compact batch 21359
20/02/20 19:27:39 WARN FileStreamSinkLog: Loaded 1068000 entries (397985432 bytes in memory), and wrote 1068000 entries for compact batch 21359
20/02/20 19:29:52 WARN FileStreamSourceLog: Compacting took 3777 ms (load: 1524 ms, write: 2253 ms) for compact batch 21369
20/02/20 19:29:52 WARN FileStreamSourceLog: Loaded 229477 entries (68970112 bytes in memory), and wrote 229477 entries for compact batch 21369
20/02/20 19:30:17 WARN FileStreamSinkLog: Compacting took 24183 ms (load: 12992 ms, write: 11191 ms) for compact batch 21369
20/02/20 19:30:20 WARN FileStreamSinkLog: Loaded 1068500 entries (398171880 bytes in memory), and wrote 1068500 entries for compact batch 21369
...
```

![Screen Shot 2020-02-21 at 12 34 22 PM](https://user-images.githubusercontent.com/1317309/75002142-c6830100-54a6-11ea-8da6-17afb056653b.png)

This messages are explaining why the operation duration peaks per every 10 batches which is compact interval. Latency from addBatch heavily increases in each peak which DOES NOT mean it takes more time to write outputs, but we have no idea if such message is not presented.

NOTE: The output may be a bit different from the code, as it may be changed a bit during review phase.

Closes #27557 from HeartSaVioR/SPARK-30804.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-24 12:34:44 +09:00
Max Gekk 26165427c7 [SPARK-31488][SQL] Support java.time.LocalDate in Parquet filter pushdown
### What changes were proposed in this pull request?
1. Modified `ParquetFilters.valueCanMakeFilterOn()` to accept filters with `java.time.LocalDate` attributes.
2. Modified `ParquetFilters.dateToDays()` to support both types `java.sql.Date` and `java.time.LocalDate` in conversions to days.
3. Add implicit conversion from `LocalDate` to `Expression` (`Literal`).

### Why are the changes needed?
To support pushed down filters with `java.time.LocalDate` attributes. Before the changes, date filters are not pushed down to Parquet datasource when `spark.sql.datetime.java8API.enabled` is `true`.

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

### How was this patch tested?
Added a test to `ParquetFilterSuite`

Closes #28259 from MaxGekk/parquet-filter-java8-date-time.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-24 02:21:53 +00:00
Takeshi Yamamuro 42f496f6ac [SPARK-31526][SQL][TESTS] Add a new test suite for ExpressionInfo
### What changes were proposed in this pull request?

This PR intends to add a new test suite for `ExpressionInfo`. Major changes are as follows;

 - Added a new test suite named `ExpressionInfoSuite`
 - To improve test coverage, added a test for error handling in `ExpressionInfoSuite`
 - Moved the `ExpressionInfo`-related tests from `UDFSuite` to `ExpressionInfoSuite`
 - Moved the related tests from `SQLQuerySuite` to `ExpressionInfoSuite`
 - Added a comment in `ExpressionInfoSuite` (followup of https://github.com/apache/spark/pull/28224)

### Why are the changes needed?

To improve test suites/coverage.

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

No.

### How was this patch tested?

Added tests.

Closes #28308 from maropu/SPARK-31526.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-24 11:19:20 +09:00
Takeshi Yamamuro 820733aee2 [SPARK-31476][SQL][FOLLOWUP] Add tests for extract('field', source)
### What changes were proposed in this pull request?

SPARK-31476 has supported `extract('field', source)` as side-effect, so this PR intends to add some tests for the function in `SQLQueryTestSuite`.

### Why are the changes needed?

For better test coverage.

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

No.

### How was this patch tested?

Added tests.

Closes #28276 from maropu/SPARK-31476-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-23 04:59:59 +00:00
Gabor Somogyi c619990c1d [SPARK-31272][SQL] Support DB2 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 DB2 support (other supported databases will come in later PRs).

What this PR contains:
* Added `DB2ConnectionProvider`
* Added `DB2ConnectionProviderSuite`
* Added `DB2KrbIntegrationSuite` docker integration test
* Changed DB2 JDBC driver to use the latest (test scope only)
* Changed test table data type to a type which is supported by all the databases
* Removed double connection creation on test side
* Increased connection timeout in docker tests because DB2 docker takes quite a time to start

### 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 DB2 using kerberos.

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

Closes #28215 from gaborgsomogyi/SPARK-31272.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@apache.org>
2020-04-22 17:10:30 -07:00
yi.wu 8fbfdb38c0 [SPARK-31495][SQL] Support formatted explain for AQE
### What changes were proposed in this pull request?

To support formatted explain for AQE.

### Why are the changes needed?

AQE does not support formatted explain yet. It's good to support it for better user experience, debugging, etc.

Before:
```
== Physical Plan ==
AdaptiveSparkPlan (1)
+- * HashAggregate (unknown)
   +- CustomShuffleReader (unknown)
      +- ShuffleQueryStage (unknown)
         +- Exchange (unknown)
            +- * HashAggregate (unknown)
               +- * Project (unknown)
                  +- * BroadcastHashJoin Inner BuildRight (unknown)
                     :- * LocalTableScan (unknown)
                     +- BroadcastQueryStage (unknown)
                        +- BroadcastExchange (unknown)
                           +- LocalTableScan (unknown)

(1) AdaptiveSparkPlan
Output [4]: [k#7, count(v1)#32L, sum(v1)#33L, avg(v2)#34]
Arguments: HashAggregate(keys=[k#7], functions=[count(1), sum(cast(v1#8 as bigint)), avg(cast(v2#19 as bigint))]), AdaptiveExecutionContext(org.apache.spark.sql.SparkSession104ab57b), [PlanAdaptiveSubqueries(Map())], false
```

After:
```
== Physical Plan ==
 AdaptiveSparkPlan (14)
 +- * HashAggregate (13)
    +- CustomShuffleReader (12)
       +- ShuffleQueryStage (11)
          +- Exchange (10)
             +- * HashAggregate (9)
                +- * Project (8)
                   +- * BroadcastHashJoin Inner BuildRight (7)
                      :- * Project (2)
                      :  +- * LocalTableScan (1)
                      +- BroadcastQueryStage (6)
                         +- BroadcastExchange (5)
                            +- * Project (4)
                               +- * LocalTableScan (3)

 (1) LocalTableScan [codegen id : 2]
 Output [2]: [_1#x, _2#x]
 Arguments: [_1#x, _2#x]

 (2) Project [codegen id : 2]
 Output [2]: [_1#x AS k#x, _2#x AS v1#x]
 Input [2]: [_1#x, _2#x]

 (3) LocalTableScan [codegen id : 1]
 Output [2]: [_1#x, _2#x]
 Arguments: [_1#x, _2#x]

 (4) Project [codegen id : 1]
 Output [2]: [_1#x AS k#x, _2#x AS v2#x]
 Input [2]: [_1#x, _2#x]

 (5) BroadcastExchange
 Input [2]: [k#x, v2#x]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#x]

 (6) BroadcastQueryStage
 Output [2]: [k#x, v2#x]
 Arguments: 0

 (7) BroadcastHashJoin [codegen id : 2]
 Left keys [1]: [k#x]
 Right keys [1]: [k#x]
 Join condition: None

 (8) Project [codegen id : 2]
 Output [3]: [k#x, v1#x, v2#x]
 Input [4]: [k#x, v1#x, k#x, v2#x]

 (9) HashAggregate [codegen id : 2]
 Input [3]: [k#x, v1#x, v2#x]
 Keys [1]: [k#x]
 Functions [3]: [partial_count(1), partial_sum(cast(v1#x as bigint)), partial_avg(cast(v2#x as bigint))]
 Aggregate Attributes [4]: [count#xL, sum#xL, sum#x, count#xL]
 Results [5]: [k#x, count#xL, sum#xL, sum#x, count#xL]

 (10) Exchange
 Input [5]: [k#x, count#xL, sum#xL, sum#x, count#xL]
 Arguments: hashpartitioning(k#x, 5), true, [id=#x]

 (11) ShuffleQueryStage
 Output [5]: [sum#xL, k#x, sum#x, count#xL, count#xL]
 Arguments: 1

 (12) CustomShuffleReader
 Input [5]: [k#x, count#xL, sum#xL, sum#x, count#xL]
 Arguments: coalesced

 (13) HashAggregate [codegen id : 3]
 Input [5]: [k#x, count#xL, sum#xL, sum#x, count#xL]
 Keys [1]: [k#x]
 Functions [3]: [count(1), sum(cast(v1#x as bigint)), avg(cast(v2#x as bigint))]
 Aggregate Attributes [3]: [count(1)#xL, sum(cast(v1#x as bigint))#xL, avg(cast(v2#x as bigint))#x]
 Results [4]: [k#x, count(1)#xL AS count(v1)#xL, sum(cast(v1#x as bigint))#xL AS sum(v1)#xL, avg(cast(v2#x as bigint))#x AS avg(v2)#x]

 (14) AdaptiveSparkPlan
 Output [4]: [k#x, count(v1)#xL, sum(v1)#xL, avg(v2)#x]
 Arguments: isFinalPlan=true
```

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

No, this should be new feature along with AQE in Spark 3.0.

### How was this patch tested?

Added a query file: `explain-aqe.sql` and a unit test.

Closes #28271 from Ngone51/support_formatted_explain_for_aqe.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-22 12:44:06 +00:00
Kent Yao 37d2e037ed [SPARK-31507][SQL] Remove uncommon fields support and update some fields with meaningful names for extract function
### What changes were proposed in this pull request?

Extracting millennium, century, decade, millisecond, microsecond and epoch from datetime is neither ANSI standard nor quite common in modern SQL platforms. Most of the systems listing below does not support these except PostgreSQL and redshift.

https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF

https://docs.oracle.com/cd/B19306_01/server.102/b14200/functions050.htm

https://prestodb.io/docs/current/functions/datetime.html

https://docs.cloudera.com/documentation/enterprise/5-8-x/topics/impala_datetime_functions.html

https://docs.snowflake.com/en/sql-reference/functions-date-time.html#label-supported-date-time-parts

https://www.postgresql.org/docs/9.1/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT

This PR removes these extract fields support from extract function for date and timestamp values

`isoyear` is PostgreSQL specific but `yearofweek` is more commonly used across platforms
`isodow` is PostgreSQL specific but `iso` as a suffix is more commonly used across platforms so, `dow_iso` and `dayofweek_iso` is used to replace it.

For historical reasons, we have [`dayofweek`, `dow`] implemented for representing a non-ISO day-of-week and a newly added `isodow` from PostgreSQL for ISO day-of-week. Many other systems only have one week-numbering system support and use either full names or abbreviations. Things in spark become a little bit complicated.
1. because of the existence of `isodow`, so we need to add iso-prefix to `dayofweek` to make a pair for it too. [`dayofweek`, `isodayofweek`, `dow` and `isodow`]
2. because there are rare `iso`-prefixed systems and more systems choose `iso`-suffixed way, so we may result in [`dayofweek`, `dayofweekiso`, `dow`, `dowiso`]
3. `dayofweekiso` looks nice and has use cases in the platforms listed above, e.g. snowflake, but `dowiso` looks weird and no use cases found.
4. with a discussion the community,we have agreed with an underscore before `iso` may look much better because `isodow` is new and there is no standard for `iso` kind of things, so this may be good for us to make it simple and clear for end-users if they are well documented too.

Thus, we finally result in [`dayofweek`, `dow`] for Non-ISO day-of-week system and [`dayofweek_iso`, `dow_iso`] for ISO system

### Why are the changes needed?

Remove some nonstandard and uncommon features as we can add them back if necessary

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

NO, we should target this to 3.0.0 and these are added during 3.0.0

### How was this patch tested?

Remove unused tests

Closes #28284 from yaooqinn/SPARK-31507.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-22 10:24:49 +00:00
Kent Yao 2c2062ea7c [SPARK-31498][SQL][DOCS] Dump public static sql configurations through doc generation
### What changes were proposed in this pull request?

Currently, only the non-static public SQL configurations are dump to public doc, we'd better also add those static public ones as the command `set -v`

This PR force call StaticSQLConf to buildStaticConf.

### Why are the changes needed?

Fix missing SQL configurations in doc

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

NO

### How was this patch tested?

add unit test and verify locally to see if public static SQL conf is in `docs/sql-config.html`

Closes #28274 from yaooqinn/SPARK-31498.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-22 10:16:39 +00:00
herman cf6038499d
[SPARK-31511][SQL] Make BytesToBytesMap iterators thread-safe
### What changes were proposed in this pull request?
This PR increases the thread safety of the `BytesToBytesMap`:
- It makes the `iterator()` and `destructiveIterator()` methods used their own `Location` object. This used to be shared, and this was causing issues when the map was being iterated over in two threads by two different iterators.
- Removes the `safeIterator()` function. This is not needed anymore.
- Improves the documentation of a couple of methods w.r.t. thread-safety.

### Why are the changes needed?
It is unexpected an iterator shares the object it is returning with all other iterators. This is a violation of the iterator contract, and it causes issues with iterators over a map that are consumed in different threads.

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

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

Closes #28286 from hvanhovell/SPARK-31511.

Authored-by: herman <herman@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-21 18:17:19 -07:00
Wenchen Fan b209b5f406
[SPARK-31503][SQL] fix the SQL string of the TRIM functions
### What changes were proposed in this pull request?

override the `sql` method of `StringTrim`, `StringTrimLeft` and `StringTrimRight`, to use the standard SQL syntax.

### Why are the changes needed?

The current implementation is wrong. It gives you a SQL string that returns different result.

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

No

### How was this patch tested?

new tests

Closes #28281 from cloud-fan/sql.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-21 11:22:18 -07:00
Wenchen Fan a5ebbacf53 [SPARK-31361][SQL] Rebase datetime in parquet/avro according to file metadata
### What changes were proposed in this pull request?

This PR adds a new parquet/avro file metadata: `org.apache.spark.legacyDatetime`. It indicates that the file was written with the "rebaseInWrite" config enabled, and spark need to do rebase when reading it.

This makes Spark be able to do rebase more smartly:
1. If we don't know which Spark version writes the file, do rebase if the "rebaseInRead" config is true.
2. If the file was written by Spark 2.4 and earlier, then do rebase.
3. If the file was written by Spark 3.0 and later, do rebase if the `org.apache.spark.legacyDatetime` exists in file metadata.

### Why are the changes needed?

It's very easy to have mixed-calendar parquet/avro files: e.g. A user upgrades to Spark 3.0 and writes some parquet files to an existing directory. Then he realizes that the directory contains legacy datetime values before 1582. However, it's too late and he has to find out all the legacy files manually and read them separately.

To support mixed-calendar parquet/avro files, we need to decide to rebase or not based on the file metadata.

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

No

### How was this patch tested?

Updated test

Closes #28137 from cloud-fan/datetime.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-22 00:26:23 +09:00
yi.wu 55b026a783 [SPARK-31504][SQL] Formatted Explain should have determined order of Output fields
### What changes were proposed in this pull request?

In `verboseStringWithOperatorId`, use `output` (it's `Seq[Attribute]`) instead of `producedAttributes` (it's `AttributeSet`) to generates `"Output"` for the leaf node in order to make `"Output"` determined.

### Why are the changes needed?

Currently, Formatted Explain use `producedAttributes`, the `AttributeSet`,  to generate `"Output"`. As a result, the fields order within `"Output"` can be different from time to time. It's That means, for the same plan, it could have different explain outputs.

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

Yes, user see the determined fields order within formatted explain now.

### How was this patch tested?

Added a regression test.

Closes #28282 from Ngone51/fix_output.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-21 12:33:58 +00:00
Kent Yao 1985437110 [SPARK-31474][SQL] Consistency between dayofweek/dow in extract exprsession and dayofweek function
### What changes were proposed in this pull request?
```sql
spark-sql> SELECT extract(dayofweek from '2009-07-26');
1
spark-sql> SELECT extract(dow from '2009-07-26');
0
spark-sql> SELECT extract(isodow from '2009-07-26');
7
spark-sql> SELECT dayofweek('2009-07-26');
1
spark-sql> SELECT weekday('2009-07-26');
6
```
Currently, there are 4 types of day-of-week range:
1. the function `dayofweek`(2.3.0) and extracting `dayofweek`(2.4.0) result as of Sunday(1) to Saturday(7)
2. extracting `dow`(3.0.0) results as of Sunday(0) to Saturday(6)
3. extracting` isodow` (3.0.0) results as of Monday(1) to Sunday(7)
4. the function `weekday`(2.4.0) results as of Monday(0) to Sunday(6)

Actually, extracting `dayofweek` and `dow` are both derived from PostgreSQL but have different meanings.
https://issues.apache.org/jira/browse/SPARK-23903
https://issues.apache.org/jira/browse/SPARK-28623

In this PR, we make extracting `dow` as same as extracting `dayofweek` and the `dayofweek` function for historical reason and not breaking anything.

Also, add more documentation to the extracting function to make extract field more clear to understand.

### Why are the changes needed?

Consistency insurance

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

yes, doc updated and extract `dow` is as same as `dayofweek`

### How was this patch tested?

1. modified ut
2. local SQL doc verification
#### before
![image](https://user-images.githubusercontent.com/8326978/79601949-3535b100-811c-11ea-957b-a33d68641181.png)

#### after
![image](https://user-images.githubusercontent.com/8326978/79601847-12a39800-811c-11ea-8ff6-aa329255d099.png)

Closes #28248 from yaooqinn/SPARK-31474.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-21 11:55:33 +00:00
Maryann Xue ae29cf24fc [SPARK-31501][SQL] AQE update UI should not cause deadlock
### What changes were proposed in this pull request?

This PR makes sure that AQE does not call update UI if the current execution ID does not match the current query. This PR also includes a minor refactoring that moves `getOrCloneSessionWithAqeOff` from `QueryExecution` to `AdaptiveSparkPlanHelper` since that function is not used by `QueryExecution` any more.

### Why are the changes needed?

Without this fix, there could be a potential deadlock.

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

No.

### How was this patch tested?

Added UT.

Closes #28275 from maryannxue/aqe-ui-deadlock.

Authored-by: Maryann Xue <maryann.xue@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-21 03:56:42 +00:00
Takeshi Yamamuro e42dbe7cd4 [SPARK-31429][SQL][DOC] Automatically generates a SQL document for built-in functions
### What changes were proposed in this pull request?

This PR intends to add a Python script to generates a SQL document for built-in functions and the document in SQL references.

### Why are the changes needed?

To make SQL references complete.

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

Yes;

![a](https://user-images.githubusercontent.com/692303/79406712-c39e1b80-7fd2-11ea-8b85-9f9cbb6efed3.png)
![b](https://user-images.githubusercontent.com/692303/79320526-eb46a280-7f44-11ea-8639-90b1fb2b8848.png)
![c](https://user-images.githubusercontent.com/692303/79320707-3365c500-7f45-11ea-9984-69ffe800fb87.png)

### How was this patch tested?

Manually checked and added tests.

Closes #28224 from maropu/SPARK-31429.

Lead-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-21 10:55:13 +09:00
rishi 4f8b03d336
[SPARK-31389][SQL][TESTS] Add codegen-on test coverage for some tests in SQLMetricsSuite
### What changes were proposed in this pull request?
Adding missing unit tests in SQLMetricSuite to cover the code generated path.
**Additional tests were added in the following unit tests.**
Filter metrics, SortMergeJoin metrics, SortMergeJoin(outer) metrics, BroadcastHashJoin metrics,  ShuffledHashJoin metrics, BroadcastHashJoin(outer) metrics, BroadcastNestedLoopJoin metrics, BroadcastLeftSemiJoinHash metrics, CartesianProduct metrics,  SortMergeJoin(left-anti) metrics

### Why are the changes needed?
The existing tests in SQLMetricSuite only cover the interpreted path.
It is necessary for the tests to cover code generated path as well since CodeGenerated path is often used in production.

The PR doesn't change test("Aggregate metrics") and test("ObjectHashAggregate metrics"). The test("Aggregate metrics") tests metrics when a HashAggregate is used. Enabling codegen forces the test to use ObjectHashAggregate rather than the regular HashAggregate. ObjectHashAggregate has a test of its own. Therefore, I feel these two tests need not enabling codegen is not necessary.

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

### How was this patch tested?
I added debug statements in the code to make sure both Code generated and Interpreted paths are being exercised.
I further used Intellij debugger to ensure that the newly added unit tests are in fact exercising both code generated and interpreted paths.

Closes #28173 from sririshindra/SPARK-31389.

Authored-by: rishi <spothireddi@cloudera.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-20 14:41:45 -07:00
Wenchen Fan 69f9ee18b6
[SPARK-31452][SQL] Do not create partition spec for 0-size partitions in AQE
### What changes were proposed in this pull request?

This PR skips creating the partition specs in `ShufflePartitionsUtil` for 0-size partitions, which avoids launching unnecessary tasks that do nothing.

### Why are the changes needed?

launching tasks that do nothing is a waste.

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

no

### How was this patch tested?

updated tests

Closes #28226 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-20 13:50:07 -07:00
gatorsmile 6c792a79c1 [SPARK-31234][SQL][FOLLOW-UP] ResetCommand should not affect static SQL Configuration
### What changes were proposed in this pull request?
This PR is the follow-up PR of https://github.com/apache/spark/pull/28003

- add a migration guide
- add an end-to-end test case.

### Why are the changes needed?
The original PR made the major behavior change in the user-facing RESET command.

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

### How was this patch tested?
Added a new end-to-end test

Closes #28265 from gatorsmile/spark-31234followup.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2020-04-20 13:08:55 -07:00
Maryann Xue 44d370dd45 [SPARK-31475][SQL] Broadcast stage in AQE did not timeout
### What changes were proposed in this pull request?

This PR adds a timeout for the Future of a BroadcastQueryStageExec to make sure it can have the same timeout behavior as a non-AQE broadcast exchange.

### Why are the changes needed?

This is to make the broadcast timeout behavior in AQE consistent with that in non-AQE.

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

No.

### How was this patch tested?

Added UT.

Closes #28250 from maryannxue/aqe-broadcast-timeout.

Authored-by: Maryann Xue <maryann.xue@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2020-04-20 11:55:48 -07:00
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