Commit graph

6863 commits

Author SHA1 Message Date
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