Commit graph

30575 commits

Author SHA1 Message Date
Max Gekk 37ef7bb98c [SPARK-35840][SQL] Add apply() for a single field to YearMonthIntervalType and DayTimeIntervalType
### What changes were proposed in this pull request?
In the PR, I propose to add 2 new methods that accept one field and produce either `YearMonthIntervalType` or `DayTimeIntervalType`.

### Why are the changes needed?
To improve code maintenance.

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

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

Closes #32997 from MaxGekk/ansi-interval-types-single-field.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-21 14:15:33 +03:00
Angerszhuuuu 1488ea9a8c [SPARK-35820][SQL] Support Cast between different field DayTimeIntervalType
### What changes were proposed in this pull request?
 Support Cast between different field DayTimeIntervalType

### Why are the changes needed?
Make user convenient to get different field DayTimeIntervalType

### Does this PR introduce _any_ user-facing change?
User can call cast DayTimeIntervalType(DAY, SECOND) to DayTimeIntervalType(DAY, MINUTE) etc

### How was this patch tested?
Added UT

Closes #32975 from AngersZhuuuu/SPARK-35820.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-21 12:36:38 +03:00
Jungtaek Lim 4a6d90e187 [SPARK-35611][SS] Introduce the strategy on mismatched offset for start offset timestamp on Kafka data source
### What changes were proposed in this pull request?

This PR proposes to introduce the strategy on mismatched offset for start offset timestamp on Kafka data source.

Please read the section `Why are the changes needed?` to understand the rationalization of the functionality.

This would be pretty much helpful for the case where there's a skew between partitions and some partitions have older records.

* AS-IS: Spark simply fails the query and end users have to deal with workarounds requiring manual steps.
* TO-BE: Spark will assign the latest offset for these partitions, so that Spark can read newer records from these partitions in further micro-batches.

To retain the existing behavior and also give some help for the proposed "TO-BE" behavior, we'd like to introduce the strategy on mismatched offset for start offset timestamp to let end users choose from them.

The strategy will be added as source option, to ensure end users set the behavior explicitly (otherwise simply "known" default value).

* New source option to be added: startingOffsetsByTimestampStrategy
* Available values: `error` (fail the query as referred as AS-IS), `latest` (set the offset to the latest as referred as TO-BE)

Doc changes are following:

![ES-106042-doc-screenshot-1](https://user-images.githubusercontent.com/1317309/120472697-2c1ba800-c3e1-11eb-884f-f28152168053.png)
![ES-106042-doc-screenshot-2](https://user-images.githubusercontent.com/1317309/120472719-33db4c80-c3e1-11eb-9851-939be8a3ddb7.png)

### Why are the changes needed?

We encountered a real-world case Spark fails the query if some of the partitions don't have matching offset by timestamp.

This is intended behavior to avoid bring unintended output for some cases like:

* timestamp 2 is presented as timestamp-offset, but the some of partitions don't have the record yet
* record with timestamp 1 comes "later" in the following micro-batch

which is possible since Kafka allows to specify the timestamp in record.

Here the unintended output we talked about was the risk of reading record with timestamp 1 in the next micro-batch despite the option specifying timestamp 2.

But for many cases end users just suppose timestamp is increasing monotonically with wall clocks are all in sync, and current behavior blocks these cases to make progress.

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

Yes, but not a breaking change. It's up to end users to choose the behavior which the default value is "error" (current behavior). And it's a source option (not config) so they need to explicitly set the behavior to let the functionality takes effect.

### How was this patch tested?

New UTs.

Closes #32747 from HeartSaVioR/SPARK-35611.

Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-06-21 00:37:42 -07:00
yi.wu 974d127c4f [SPARK-35545][FOLLOW-UP][TEST][SQL] Add a regression test for the SubqueryExpression refactor
### What changes were proposed in this pull request?

Add a test.

### Why are the changes needed?

The SubqueryExpression refactor PR https://github.com/apache/spark/pull/32687 actually fixes the bug of `SubqueryExpression.references`. So this follow-up PR adds a regression unit test for it.

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

No.

### How was this patch tested?

Added a new test.

Closes #32990 from Ngone51/spark-35545-followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-21 09:54:55 +03:00
Peter Toth 682e7f2033 [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse
### What changes were proposed in this pull request?
This PR:
1. Fixes an issue in `ReuseExchange` rule that can result a `ReusedExchange` node pointing to an invalid exchange. This can happen due to the 2 separate traversals in `ReuseExchange` when the 2nd traversal modifies an exchange that has already been referenced (reused) in the 1st traversal.
   Consider the following query:
   ```
   WITH t AS (
     SELECT df1.id, df2.k
     FROM df1 JOIN df2 ON df1.k = df2.k
     WHERE df2.id < 2
   )
   SELECT * FROM t AS a JOIN t AS b ON a.id = b.id
   ```
   Before this PR the plan of the query was (note the `<== this reuse node points to a non-existing node` marker):
   ```
   == Physical Plan ==
   *(7) SortMergeJoin [id#14L], [id#18L], Inner
   :- *(3) Sort [id#14L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#14L, 5), true, [id=#298]
   :     +- *(2) Project [id#14L, k#17L]
   :        +- *(2) BroadcastHashJoin [k#15L], [k#17L], Inner, BuildRight
   :           :- *(2) Project [id#14L, k#15L]
   :           :  +- *(2) Filter isnotnull(id#14L)
   :           :     +- *(2) ColumnarToRow
   :           :        +- FileScan parquet default.df1[id#14L,k#15L] Batched: true, DataFilters: [isnotnull(id#14L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#15L), dynamicpruningexpression(k#15L IN dynamicpruning#26)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :           :              +- SubqueryBroadcast dynamicpruning#26, 0, [k#17L], [id=#289]
   :           :                 +- ReusedExchange [k#17L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#179]
   :           +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#179]
   :              +- *(1) Project [k#17L]
   :                 +- *(1) Filter ((isnotnull(id#16L) AND (id#16L < 2)) AND isnotnull(k#17L))
   :                    +- *(1) ColumnarToRow
   :                       +- FileScan parquet default.df2[id#16L,k#17L] Batched: true, DataFilters: [isnotnull(id#16L), (id#16L < 2), isnotnull(k#17L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint>
   +- *(6) Sort [id#18L ASC NULLS FIRST], false, 0
      +- ReusedExchange [id#18L, k#21L], Exchange hashpartitioning(id#14L, 5), true, [id=#184] <== this reuse node points to a non-existing node
   ```
   After this PR:
   ```
   == Physical Plan ==
   *(7) SortMergeJoin [id#14L], [id#18L], Inner
   :- *(3) Sort [id#14L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#14L, 5), true, [id=#231]
   :     +- *(2) Project [id#14L, k#17L]
   :        +- *(2) BroadcastHashJoin [k#15L], [k#17L], Inner, BuildRight
   :           :- *(2) Project [id#14L, k#15L]
   :           :  +- *(2) Filter isnotnull(id#14L)
   :           :     +- *(2) ColumnarToRow
   :           :        +- FileScan parquet default.df1[id#14L,k#15L] Batched: true, DataFilters: [isnotnull(id#14L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#15L), dynamicpruningexpression(k#15L IN dynamicpruning#26)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :           :              +- SubqueryBroadcast dynamicpruning#26, 0, [k#17L], [id=#103]
   :           :                 +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#102]
   :           :                    +- *(1) Project [k#17L]
   :           :                       +- *(1) Filter ((isnotnull(id#16L) AND (id#16L < 2)) AND isnotnull(k#17L))
   :           :                          +- *(1) ColumnarToRow
   :           :                             +- FileScan parquet default.df2[id#16L,k#17L] Batched: true, DataFilters: [isnotnull(id#16L), (id#16L < 2), isnotnull(k#17L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint>
   :           +- ReusedExchange [k#17L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#102]
   +- *(6) Sort [id#18L ASC NULLS FIRST], false, 0
      +- ReusedExchange [id#18L, k#21L], Exchange hashpartitioning(id#14L, 5), true, [id=#231]
   ```
2. Fixes an issue with separate consecutive `ReuseExchange` and `ReuseSubquery` rules that can result a `ReusedExchange` node pointing to an invalid exchange. This can happen due to the 2 separate rules when `ReuseSubquery` rule modifies an exchange that has already been referenced (reused) in `ReuseExchange` rule.
   Consider the following query:
   ```
   WITH t AS (
     SELECT df1.id, df2.k
     FROM df1 JOIN df2 ON df1.k = df2.k
     WHERE df2.id < 2
   ),
   t2 AS (
     SELECT * FROM t
     UNION
     SELECT * FROM t
   )
   SELECT * FROM t2 AS a JOIN t2 AS b ON a.id = b.id
   ```
   Before this PR the plan of the query was (note the `<== this reuse node points to a non-existing node` marker):
   ```
   == Physical Plan ==
   *(15) SortMergeJoin [id#46L], [id#58L], Inner
   :- *(7) Sort [id#46L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#46L, 5), true, [id=#979]
   :     +- *(6) HashAggregate(keys=[id#46L, k#49L], functions=[])
   :        +- Exchange hashpartitioning(id#46L, k#49L, 5), true, [id=#975]
   :           +- *(5) HashAggregate(keys=[id#46L, k#49L], functions=[])
   :              +- Union
   :                 :- *(2) Project [id#46L, k#49L]
   :                 :  +- *(2) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight
   :                 :     :- *(2) Project [id#46L, k#47L]
   :                 :     :  +- *(2) Filter isnotnull(id#46L)
   :                 :     :     +- *(2) ColumnarToRow
   :                 :     :        +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :                 :     :              +- SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#926]
   :                 :     :                 +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#656]
   :                 :     +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#656]
   :                 :        +- *(1) Project [k#49L]
   :                 :           +- *(1) Filter ((isnotnull(id#48L) AND (id#48L < 2)) AND isnotnull(k#49L))
   :                 :              +- *(1) ColumnarToRow
   :                 :                 +- FileScan parquet default.df2[id#48L,k#49L] Batched: true, DataFilters: [isnotnull(id#48L), (id#48L < 2), isnotnull(k#49L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint>
   :                 +- *(4) Project [id#46L, k#49L]
   :                    +- *(4) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight
   :                       :- *(4) Project [id#46L, k#47L]
   :                       :  +- *(4) Filter isnotnull(id#46L)
   :                       :     +- *(4) ColumnarToRow
   :                       :        +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :                       :              +- ReusedSubquery SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#926]
   :                       +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#656]
   +- *(14) Sort [id#58L ASC NULLS FIRST], false, 0
      +- ReusedExchange [id#58L, k#61L], Exchange hashpartitioning(id#46L, 5), true, [id=#761] <== this reuse node points to a non-existing node
   ```
   After this PR:
   ```
   == Physical Plan ==
   *(15) SortMergeJoin [id#46L], [id#58L], Inner
   :- *(7) Sort [id#46L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#46L, 5), true, [id=#793]
   :     +- *(6) HashAggregate(keys=[id#46L, k#49L], functions=[])
   :        +- Exchange hashpartitioning(id#46L, k#49L, 5), true, [id=#789]
   :           +- *(5) HashAggregate(keys=[id#46L, k#49L], functions=[])
   :              +- Union
   :                 :- *(2) Project [id#46L, k#49L]
   :                 :  +- *(2) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight
   :                 :     :- *(2) Project [id#46L, k#47L]
   :                 :     :  +- *(2) Filter isnotnull(id#46L)
   :                 :     :     +- *(2) ColumnarToRow
   :                 :     :        +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :                 :     :              +- SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#485]
   :                 :     :                 +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#484]
   :                 :     :                    +- *(1) Project [k#49L]
   :                 :     :                       +- *(1) Filter ((isnotnull(id#48L) AND (id#48L < 2)) AND isnotnull(k#49L))
   :                 :     :                          +- *(1) ColumnarToRow
   :                 :     :                             +- FileScan parquet default.df2[id#48L,k#49L] Batched: true, DataFilters: [isnotnull(id#48L), (id#48L < 2), isnotnull(k#49L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint>
   :                 :     +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#484]
   :                 +- *(4) Project [id#46L, k#49L]
   :                    +- *(4) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight
   :                       :- *(4) Project [id#46L, k#47L]
   :                       :  +- *(4) Filter isnotnull(id#46L)
   :                       :     +- *(4) ColumnarToRow
   :                       :        +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :                       :              +- ReusedSubquery SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#485]
   :                       +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#484]
   +- *(14) Sort [id#58L ASC NULLS FIRST], false, 0
      +- ReusedExchange [id#58L, k#61L], Exchange hashpartitioning(id#46L, 5), true, [id=#793]
   ```
   (This example contains issue 1 as well.)

3. Improves the reuse of exchanges and subqueries by enabling reuse across the whole plan. This means that the new combined rule utilizes the reuse opportunities between parent and subqueries by traversing the whole plan. The traversal is started on the top level query only.

4. Due to the order of traversal this PR does while adding reuse nodes, the reuse nodes appear in parent queries if reuse is possible between different levels of queries (typical for DPP). This is not an issue from execution perspective, but this also means "forward references" in explain formatted output where parent queries come first. The changes I made to `ExplainUtils` are to handle these references properly.

This PR fixes the above 3 issues by unifying the separate rules into a `ReuseExchangeAndSubquery` rule that does a 1 pass, whole-plan, bottom-up traversal.

### Why are the changes needed?
Performance improvement.

### How was this patch tested?
- New UTs in `ReuseExchangeAndSubquerySuite` to cover 1. and 2.
- New UTs in `DynamicPartitionPruningSuite`, `SubquerySuite` and `ExchangeSuite` to cover 3.
- New `ReuseMapSuite` to test `ReuseMap`.
- Checked new golden files of `PlanStabilitySuite`s for invalid reuse references.
- TPCDS benchmarks.

Closes #28885 from peter-toth/SPARK-29375-SPARK-28940-whole-plan-reuse.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-21 04:53:19 +00:00
Hyukjin Kwon 248fda3ead [SPARK-35834][PYTHON] Use the same cleanup logic as Py4J in inheritable thread API
### What changes were proposed in this pull request?

This PR fixes the cleanup logic in inheritable thread API by following Py4J cleanup logic at https://github.com/bartdag/py4j/blob/master/py4j-python/src/py4j/clientserver.py#L269-L278.

Currently the tests that use `inheritable_thread_target` are flaky (https://github.com/apache/spark/runs/2870944288):

```
======================================================================
ERROR [71.813s]: test_save_load_pipeline_estimator (pyspark.ml.tests.test_tuning.CrossValidatorTests)
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/__w/spark/spark/python/pyspark/ml/tests/test_tuning.py", line 589, in test_save_load_pipeline_estimator
    self._run_test_save_load_pipeline_estimator(DummyLogisticRegression)
  File "/__w/spark/spark/python/pyspark/ml/tests/test_tuning.py", line 572, in _run_test_save_load_pipeline_estimator
    cvModel2 = crossval2.fit(training)
  File "/__w/spark/spark/python/pyspark/ml/base.py", line 161, in fit
    return self._fit(dataset)
  File "/__w/spark/spark/python/pyspark/ml/tuning.py", line 747, in _fit
    bestModel = est.fit(dataset, epm[bestIndex])
  File "/__w/spark/spark/python/pyspark/ml/base.py", line 159, in fit
    return self.copy(params)._fit(dataset)
  File "/__w/spark/spark/python/pyspark/ml/pipeline.py", line 114, in _fit
    model = stage.fit(dataset)
  File "/__w/spark/spark/python/pyspark/ml/base.py", line 161, in fit
    return self._fit(dataset)
  File "/__w/spark/spark/python/pyspark/ml/pipeline.py", line 114, in _fit
    model = stage.fit(dataset)
  File "/__w/spark/spark/python/pyspark/ml/base.py", line 161, in fit
    return self._fit(dataset)
  File "/__w/spark/spark/python/pyspark/ml/classification.py", line 2924, in _fit
    models = pool.map(inheritable_thread_target(trainSingleClass), range(numClasses))
  File "/__t/Python/3.6.13/x64/lib/python3.6/multiprocessing/pool.py", line 266, in map
    return self._map_async(func, iterable, mapstar, chunksize).get()
  File "/__t/Python/3.6.13/x64/lib/python3.6/multiprocessing/pool.py", line 644, in get
    raise self._value
  File "/__t/Python/3.6.13/x64/lib/python3.6/multiprocessing/pool.py", line 119, in worker
    result = (True, func(*args, **kwds))
  File "/__t/Python/3.6.13/x64/lib/python3.6/multiprocessing/pool.py", line 44, in mapstar
    return list(map(*args))
  File "/__w/spark/spark/python/pyspark/util.py", line 324, in wrapped
    InheritableThread._clean_py4j_conn_for_current_thread()
  File "/__w/spark/spark/python/pyspark/util.py", line 389, in _clean_py4j_conn_for_current_thread
    del connections[i]
IndexError: deque index out of range

----------------------------------------------------------------------
```

This seems to be because the connection deque `jvm._gateway_client.deque` is accessed, and modified by other threads. Therefore, the number of threads could be changed in the middle. Using `SparkContext._lock` doesn't protect because the deque can be updated for every Java instance access in Py4J.

This PR proposes to use the atomic `deque.remove` in the problematic dequeue alone with try-catch on `ValueError` in case it's [deleted by Py4J](https://github.com/bartdag/py4j/blob/master/py4j-python/src/py4j/clientserver.py#L269-L278).

### Why are the changes needed?

To fix the flakiness in the tests, and avoid possible breakage in user application by using this API.

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

If users were dependent on InheritableThread with pinned thread mode on, they might have faced such issues intermittently. This PR fixes it.

### How was this patch tested?

Manually tested. CI should test it out too.

Closes #32989 from HyukjinKwon/SPARK-35834.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-21 12:00:16 +09:00
Kevin Su 653be9d774 [SPARK-35811][PYTHON] Deprecate DataFrame.to_spark_io
### What changes were proposed in this pull request?

Deprecate the `DataFrame.to_spark_io`

### Why are the changes needed?

We should deprecate the `DataFrame.to_spark_io` since it's duplicated with `DataFrame.spark.to_spark_io`, and it's not existed in pandas.

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

Yes, users will get warning while using `DataFrame.to_spark_io` api.

### How was this patch tested?

Pass the CIs

Closes #32964 from pingsutw/SPARK-35811.

Authored-by: Kevin Su <pingsutw@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-21 10:43:34 +09:00
Chandni Singh 8ce1e344e5 [SPARK-35671][SHUFFLE][CORE] Add support in the ESS to serve merged shuffle block meta and data to executors
### What changes were proposed in this pull request?
This adds support in the ESS to serve merged shuffle block meta and data requests to executors.
This change is needed for fetching remote merged shuffle data from the remote shuffle services. This is part of push-based shuffle SPIP [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602).

This change introduces new messages between clients and the external shuffle service:

1. `MergedBlockMetaRequest`: The client sends this to external shuffle to get the meta information for a merged block. The response to this is one of these :
  - `MergedBlockMetaSuccess` : contains request id, number of chunks, and a `ManagedBuffer` which is a `FileSegmentBuffer` backed by the merged block meta file.
  - `RpcFailure`: this is sent back to client in case of failure. This is an existing message.

2. `FetchShuffleBlockChunks`: This is similar to `FetchShuffleBlocks` message but it is to fetch merged shuffle chunks instead of blocks.

### Why are the changes needed?
These changes are needed for push-based shuffle. Refer to the SPIP in [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602).

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

### How was this patch tested?
Added unit tests.
The reference PR with the consolidated changes covering the complete implementation is also provided in [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602).
We have already verified the functionality and the improved performance as documented in the SPIP doc.

Lead-authored-by: Chandni Singh chsinghlinkedin.com
Co-authored-by: Min Shen mshenlinkedin.com

Closes #32811 from otterc/SPARK-35671.

Lead-authored-by: Chandni Singh <singh.chandni@gmail.com>
Co-authored-by: Min Shen <mshen@linkedin.com>
Co-authored-by: Chandni Singh <chsingh@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-06-20 17:22:37 -05:00
Kousuke Saruta af20474c67 [SPARK-35827][SQL] Show proper error message when update column types to year-month/day-time interval
### What changes were proposed in this pull request?

This PR fixes error message shown when changing a column type to year-month/day-time interval type is attempted.

### Why are the changes needed?

It's for consistent behavior.
Updating column types to interval types are prohibited for V2 source tables.
So, if we attempt to update the type of a column to the conventional interval type, an error message like `Error in query: Cannot update <table> field <column> to interval type;`.

But, for year-month/day-time interval types, another error message like `Error in query: Cannot update <table> field <column>:<type> cannot be cast to interval year;`.

You can reproduce with the following procedure.
```
$ bin/spark-sql
spark-sql> SET spark.sql.catalog.mycatalog=<a catalog implementation class>;
spark-sql> CREATE TABLE mycatalog.t1(c1 int) USING <V2 datasource implementation class>;
spark-sql> ALTER TABLE mycatalog.t1 ALTER COLUMN c1 TYPE interval year to month;
```

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

No.

### How was this patch tested?

Modified an existing test.

Closes #32978 from sarutak/err-msg-interval.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-20 23:39:46 +03:00
Dongjoon Hyun 4f51e0045e [SPARK-35832][CORE][ML][K8S][TESTS] Add LocalRootDirsTest trait
### What changes were proposed in this pull request?

To make the test suite more robust, this PR aims to add a new trait, `LocalRootDirsTest`, by refactoring `SortShuffleSuite`'s helper functions and applying it to the following:
- ShuffleNettySuite
- ShuffleOldFetchProtocolSuite
- ExternalShuffleServiceSuite
- KubernetesLocalDiskShuffleDataIOSuite
- LocalDirsSuite
- RDDCleanerSuite
- ALSCleanerSuite

In addition, this fixes a UT in `KubernetesLocalDiskShuffleDataIOSuite`.

### Why are the changes needed?

`ShuffleSuite` is extended by four classes but only `SortShuffleSuite` does the clean-up correctly.
```
ShuffleSuite
- SortShuffleSuite
- ShuffleNettySuite
- ShuffleOldFetchProtocolSuite
- ExternalShuffleServiceSuite
```

Since `KubernetesLocalDiskShuffleDataIOSuite` is looking for the other storage directory, the leftover of `ShuffleSuite` causes flakiness.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-3.2/2649/testReport/junit/org.apache.spark.shuffle/KubernetesLocalDiskShuffleDataIOSuite/recompute_is_not_blocked_by_the_recovery/
```
org.apache.spark.SparkException: Job aborted due to stage failure: task 0.0 in stage 1.0 (TID 3) had a not serializable result: org.apache.spark.ShuffleSuite$NonJavaSerializableClass
...
org.apache.spark.shuffle.KubernetesLocalDiskShuffleDataIOSuite.$anonfun$new$2(KubernetesLocalDiskShuffleDataIOSuite.scala:52)
```

For the other suites, the clean-up implementation is used but not complete. So, they are refactored to use new trait.

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

No, this is a test-only change.

### How was this patch tested?

Pass the CIs.

Closes #32986 from dongjoon-hyun/SPARK-35832.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-20 10:53:53 -07:00
Kousuke Saruta 4758dc78a2 [SPARK-35771][SQL][FOLLOWUP] IntervalUtils.toYearMonthIntervalString should consider the case year-month type is casted as month type
### What changes were proposed in this pull request?

This PR fixes an issue that `IntervalUtils.toYearMonthIntervalString` doesn't consider the case that year-month interval type is casted as month interval type.
If a year-month interval data is casted as month interval, the value of the year is multiplied by `12` and added to the value of month. For example, `INTERVAL '1-2' YEAR TO MONTH` will be `INTERVAL '14' MONTH` if  it's casted.
If this behavior is intended, it's stringified to be `'INTERVAL 14' MONTH` but currently, it will be `INTERVAL '2' MONTH`

### Why are the changes needed?

It's a bug if the behavior of cast is intended.

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

No, because this feature is not released yet.

### How was this patch tested?

Modified the tests added in SPARK-35771 (#32924).

Closes #32982 from sarutak/fix-toYearMonthIntervalString.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-20 10:32:21 +03:00
Hyukjin Kwon 6d309914df [SPARK-35303][SPARK-35498][PYTHON][FOLLOW-UP] Copy local properties when starting the thread, and use inheritable thread in the current codebase
### What changes were proposed in this pull request?

This PR is a followup of https://github.com/apache/spark/pull/32429 and https://github.com/apache/spark/pull/32644.
I was thinking about creating separate PRs but decided to include all in this PR because it shares the same context, and should be easier to review together.

This PR includes:
- Use `InheritableThread` and `inheritable_thread_target` in the current code base to prevent potential resource leak (since we enabled pinned thread mode by default now at https://github.com/apache/spark/pull/32429)
- Copy local properties when `start` at `InheritableThread` is called to mimic JVM behaviour. Previously it was copied when `InheritableThread` instance was created (related to #32644).
- https://github.com/apache/spark/pull/32429 missed one place at `inheritable_thread_target` (https://github.com/apache/spark/blob/master/python/pyspark/util.py#L308). More specifically, I missed one place that should enable pinned thread mode by default.

### Why are the changes needed?

To mimic the JVM behaviour about thread lifecycle

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

Ideally no. One possible case is that users use `InheritableThread` with pinned thread mode enabled.
In this case, the local properties will be copied when starting the thread instead of defining the `InheritableThread` object.
This is a small difference that wouldn't likely affect end users.

### How was this patch tested?

Existing tests should cover this.

Closes #32962 from HyukjinKwon/SPARK-35498-SPARK-35303.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-20 11:48:38 +09:00
Takuya UESHIN 1589d32732 [SPARK-35472][PYTHON] Fix disallow_untyped_defs mypy checks for pyspark.pandas.generic
### What changes were proposed in this pull request?

Adds more type annotations in the file `python/pyspark/pandas/generic.py` and fixes the mypy check failures.

### Why are the changes needed?

We should enable more disallow_untyped_defs mypy checks.

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

Yes.
This PR adds more type annotations in pandas APIs on Spark module, which can impact interaction with development tools for users.

### How was this patch tested?

The mypy check with a new configuration and existing tests should pass.

Closes #32957 from ueshin/issues/SPARK-35472/disallow_untyped_defs.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-20 11:48:01 +09:00
Dongjoon Hyun 9eaf678099 [SPARK-35830][TESTS] Upgrade sbt-mima-plugin to 0.9.2
### What changes were proposed in this pull request?

This PR aims to upgrade `sbt-mima-plugin` to 0.9.2 for Apache Spark 3.2.0.

### Why are the changes needed?

`sbt-mima-plugin` 0.9.2 has the following updates including `Scala 3 initial support`.
- https://github.com/lightbend/mima/releases/tag/0.9.2
- https://github.com/lightbend/mima/releases/tag/0.9.1
- https://github.com/lightbend/mima/releases/tag/0.9.0

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

No.

### How was this patch tested?

Pass the CIs. Also, I manually deleted some lines from MiMiExclusion and verified that it's detected correctly.

Closes #32981 from dongjoon-hyun/SPARK-35830.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-20 11:20:44 +09:00
Angerszhuuuu 86bcd1fba0 [SPARK-35819][SQL] Support Cast between different field YearMonthIntervalType
### What changes were proposed in this pull request?
 Support Cast between different field YearMonthIntervalType

### Why are the changes needed?
Make user convenient to get different field YearMonthIntervalType

### Does this PR introduce _any_ user-facing change?
User can call cast YearMonthIntervalType(YEAR, MONTH) to YearMonthIntervalType(YEAR, YEAR) etc

### How was this patch tested?
Added UT

Closes #32974 from AngersZhuuuu/SPARK-35819.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-19 21:43:06 +03:00
Dongjoon Hyun a39f1eadb7 [SPARK-35824][CORE][TESTS] Convert LevelDBSuite.IntKeyType from a nested class to a normal class
### What changes were proposed in this pull request?

This PR aims to promote `LevelDBSuite.IntKeyType` class to a normal class to isolate `InMemoryIteratorSuite` from `LevelDBSuite`.

### Why are the changes needed?

We have the following test suite hierarchy.
```
DBIteratorSuite
- InMemoryIteratorSuite
- LevelDBIteratorSuite
```

`DBIteratorSuite.testRefWithIntNaturalKey` depends on `LevelDBSuite` and `InMemoryIteratorSuite` derived it. `InMemoryIteratorSuite` should not depend not `LevelDB`-specific stuff. This PR will make it sure.
```
public void testRefWithIntNaturalKey() throws Exception {
  LevelDBSuite.IntKeyType i = new LevelDBSuite.IntKeyType();
...
```

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

No.

### How was this patch tested?

Pass the CIs.

```
$ build/sbt "kvstore/test"
```

Closes #32971 from dongjoon-hyun/SPARK-35824.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-19 11:36:01 -07:00
Dongjoon Hyun aab37edefc [SPARK-35593][K8S][TESTS][FOLLOWUP] Run KubernetesLocalDiskShuffleDataIOSuite on a dedicated JVM
### What changes were proposed in this pull request?

This PR aims to run `KubernetesLocalDiskShuffleDataIOSuite` on a dedicated JVM.

### Why are the changes needed?

In Jenkins environment, `KubernetesLocalDiskShuffleDataIOSuite` and `ExternalShuffleServiceSuite` currently hit issues.
- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/140019/
![Screen Shot 2021-06-19 at 10 33 20 AM](https://user-images.githubusercontent.com/9700541/122650832-d9810200-d0e9-11eb-9f2a-4fb44bb874f3.png)

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

No.

### How was this patch tested?

Pass the Jenkins.

Closes #32976 from dongjoon-hyun/SPARK-35593-3.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-19 11:23:49 -07:00
Gengliang Wang 74d647d2ca [SPARK-35825][INFRA] Increase the heap and stack size for Maven build
### What changes were proposed in this pull request?

Increase memory configuration for Maven build.
Stack size: 64MB => 128MB
Initial heap size: 1024MB => 2048MB
Maximum heap size: 1024MB => 2048MB

The SBT builds are ok so let's keep the current configuration.

### Why are the changes needed?

The jenkins jobs are unstable due to the stackoverflow errors:
 https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-3.2-jdk-11/
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/2274/

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

No

### How was this patch tested?

Jenkins test

Closes #32961 from gengliangwang/increaseXss.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-19 10:44:46 -07:00
Angerszhuuuu 2ebad72758 [SPARK-35726][SQL] Truncate java.time.Duration by fields of day-time interval type
### What changes were proposed in this pull request?
Support truncate java.time.Duration by fields of day-time interval type.

### Why are the changes needed?
To respect fields of the target day-time interval types.

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

### How was this patch tested?
Added UT

Closes #32950 from AngersZhuuuu/SPARK-35726.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-19 13:51:21 +03:00
Dongjoon Hyun 94f701587d [SPARK-35818][BUILD] Upgrade SBT to 1.5.4
### What changes were proposed in this pull request?

This PR aims to upgrade SBT to 1.5.4.

### Why are the changes needed?

SBT 1.5.4 is released 5 days ago.
- https://github.com/sbt/sbt/releases/tag/v1.5.4

This will bring the latest bug fixes like the following.

- Fixes BSP on ARM Macs by keeping JNI server socket to keep using JNI
- Fixes compiler ClassLoader list to use compilerJars.toList (For Scala 3, this drops support for 3.0.0-M2)
- Fixes undercompilation of package object causing "Symbol 'type X' is missing from the classpath"
- Fixes overcompilation with scalac -release flag
- Fixes build/exit notification not closing BSP channel
- Fixes POM file's Maven repository ID character restriction to match that of Maven

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

No.

### How was this patch tested?

Pass the CIs.

Closes #32966 from dongjoon-hyun/SPARK-35818.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-19 00:17:35 -07:00
Dongjoon Hyun b9d6473e89 [SPARK-35593][K8S][TESTS][FOLLOWUP] Increase timeout in KubernetesLocalDiskShuffleDataIOSuite
### What changes were proposed in this pull request?

This increases the timeout from 10 seconds to 60 seconds in KubernetesLocalDiskShuffleDataIOSuite to reduce the flakiness.

### Why are the changes needed?

- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/140003/testReport/

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

No.

### How was this patch tested?

Pass the CIs

Closes #32967 from dongjoon-hyun/SPARK-35593-2.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-06-19 15:22:29 +09:00
Yikun Jiang b7df75a777 [SPARK-35708][PYTHON][TEST] Add BaseTest for DataTypeOps
### What changes were proposed in this pull request?
This patch adds DataTypeOps test to check the ops is loaded as expected.

### Why are the changes needed?
When complete https://github.com/apache/spark/pull/32821, I found there are no test for DataTypeOps. There were many logic when DataTypeOps loaded, it's better to add the test to make sure interface stable.

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

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

Closes #32859 from Yikun/SPARK-XXXXX1.

Authored-by: Yikun Jiang <yikunkero@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-06-18 18:54:50 -07:00
toujours33 d015eff16d [SPARK-35796][TESTS] Fix SparkSubmitSuite failure on MacOS 10.15+
### What changes were proposed in this pull request?
Change primaryResource assertion from exact match to suffix match in case SparkSubmitSuite.`handles k8s cluster mode`

### Why are the changes needed?
When I run SparkSubmitSuite on MacOs 10.15.7, I got AssertionError for `handles k8s cluster mode` test after pr [SPARK-35691](https://issues.apache.org/jira/browse/SPARK-35691), due to `File(path).getCanonicalFile().toURI()` function  with absolute path as parameter will return path begin with `/System/Volumes/Data` on MacOs higher tha 10.15.
eg.  `/home/testjars.jar` will get `file:/System/Volumes/Data/home/testjars.jar`

In order to pass UT on MacOs higher than 10.15, we change the assertion into suffix match

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

### How was this patch tested?
1. Pass the GitHub Action
2. Manually test
    - environment: MacOs > 10.15
    - commad: `build/mvn clean install -Phadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -pl core -am -DwildcardSuites=org.apache.spark.deploy.SparkSubmitSuite -Dtest=none`
    - Test result:
        - before this pr, case failed with following exception:
        `- handles k8s cluster mode *** FAILED ***
  Some("file:/System/Volumes/Data/home/thejar.jar") was not equal to Some("file:/home/thejar.jar") (SparkSubmitSuite.scala:485)
  Analysis:
  Some(value: "file:/[System/Volumes/Data/]home/thejar.jar" -> "file:/[]home/thejar.jar")`
        - after this pr, run all test successfully

Closes #32948 from toujours33/SPARK-35796.

Authored-by: toujours33 <wangyazhi@baidu.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-18 17:48:49 -07:00
Liang-Chi Hsieh 882122d6b7 [SPARK-35565][SS] Add config for ignoring metadata directory of FileStreamSink
### What changes were proposed in this pull request?

This patch proposes to add an internal config for ignoring metadata of `FileStreamSink` when reading the output path.

### Why are the changes needed?

`FileStreamSink` produces a metadata directory which logs output files per micro-batch. When we read from the output path, Spark will look at the metadata and ignore other files not in the log.

Normally it works well. But for some use-cases, we may need to ignore the metadata when reading the output path. For example, when we change the streaming query and must to run it with new checkpoint directory, we cannot use previous metadata. If we create a new metadata too, when we read the output path later in Spark, Spark only reads the files listed in the new metadata. The files written before we use new checkpoint and metadata are ignored by Spark.

Although seems we can output to different output directory every time, but it is bad idea as we will produce many directories unnecessarily.

We need a config for ignoring the metadata of `FileStreamSink` when reading the output path.

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

Added a config for ignoring metadata of FileStreamSink when reading the output.

### How was this patch tested?

Unit tests.

Closes #32702 from viirya/ignore-metadata.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-06-19 08:20:58 +09:00
Takuya UESHIN c879510d2f [SPARK-35478][PYTHON][FOLLOWUP] Fix Jenkins' linter
### What changes were proposed in this pull request?

This is a follow-up of #32886 to fix the Jenkins' linter.

### Why are the changes needed?

The PR #32886 was mistakenly merged before Jenkins' linter passes.

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

No.

### How was this patch tested?

Closes #32965 from ueshin/issues/SPARK-35478/fup.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-18 13:52:54 -07:00
Kevin Su 3fb044e043 [SPARK-35478][PYTHON] Enable disallow_untyped_defs mypy check for pyspark.pandas.window
### What changes were proposed in this pull request?

Adds more type annotations in the file `python/pyspark/pandas/window.py` and fixes the mypy check failures.

### Why are the changes needed?

We should enable more disallow_untyped_defs mypy checks.

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

Yes.
This PR adds more type annotations in pandas APIs on the Spark module, which can impact interaction with development tools for users.

### How was this patch tested?

The mypy check with a new configuration and existing tests should pass.

Closes #32886 from pingsutw/SPARK-35478.

Authored-by: Kevin Su <pingsutw@apache.org>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-06-18 11:21:33 -07:00
Yikun Jiang f84a720fe3 [SPARK-35342][PYTHON] Introduce DecimalOps and make isnull method data-type-based
### What changes were proposed in this pull request?
- Introduce a DecimalOps for DecimalType
- Make `isnull` method data-type-based

### Why are the changes needed?
Now DecimalType, DoubleType, and FloatType data share the FractionalOps class, but DecimalType behaves differently from FloatType and DoubleType (as https://github.com/apache/spark/blob/master/python/pyspark/pandas/base.py#L987-L990), so we propose to introduce DecimalOps. The behavior difference here is caused by DecimalType could not have NaN.

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

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

### How was this patch tested?
- New added DecimalOpsTest passed
- Existing NumOpsTest passed

Closes #32821 from Yikun/SPARK-35342.

Authored-by: Yikun Jiang <yikunkero@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-06-18 10:44:35 -07:00
Yuming Wang 7be8d8a164 [SPARK-35185][SQL] Improve Distinct statistics estimation
### What changes were proposed in this pull request?

This PR improves `Distinct` statistics estimation by rewrite it to `Aggregate`.

### Why are the changes needed?

1. The current implementation will lack column statistics.
2. Some rules before the `ReplaceDistinctWithAggregate` may use it. For example: https://github.com/apache/spark/pull/31113/files#diff-11264d807efa58054cca2d220aae8fba644ee0f0f2a4722c46d52828394846efR1808

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

No.

### How was this patch tested?

Unit test.

Closes #32291 from wangyum/SPARK-35185.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-06-18 21:48:44 +08:00
ulysses-you 2c4598d02e [SPARK-35608][SQL] Support AQE optimizer side transformUpWithPruning
### What changes were proposed in this pull request?

Change `AQEPropagateEmptyRelation` from `transformUp` to `transformUpWithPruning

### Why are the changes needed?

To avoid unnecessary iteration during AQE optimizer.

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

No.

### How was this patch tested?

Pass CI.

Closes #32742 from ulysses-you/aqe-transformUpWithPruning.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-18 20:31:11 +08:00
Takuya UESHIN 2f537a838a [SPARK-35469][PYTHON] Fix disallow_untyped_defs mypy checks
### What changes were proposed in this pull request?

Adds more type annotations in the file `python/pyspark/pandas/accessors.py` and fixes the mypy check failures.

### Why are the changes needed?

We should enable more disallow_untyped_defs mypy checks.

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

Yes.
This PR adds more type annotations in pandas APIs on Spark module, which can impact interaction with development tools for users.

### How was this patch tested?

The mypy check with a new configuration and existing tests should pass.

Closes #32956 from ueshin/issues/SPARK-35469/disallow_untyped_defs.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-18 20:43:59 +09:00
Angerszhuuuu 071566caf3 [SPARK-35769][SQL] Truncate java.time.Period by fields of year-month interval type
### What changes were proposed in this pull request?
Support truncate java.time.Period by fields of year-month interval type

### Why are the changes needed?
To follow the SQL standard and respect the field restriction of the target year-month type.

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

### How was this patch tested?
Added UT

Closes #32945 from AngersZhuuuu/SPARK-35769.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-18 11:55:57 +03:00
Hyukjin Kwon ea907469bb Revert "[SPARK-35678][ML][FOLLOWUP] softmax support offset and step"
This reverts commit fdf86fd6e7.
2021-06-18 16:45:28 +09:00
Ruifeng Zheng fdf86fd6e7 [SPARK-35678][ML][FOLLOWUP] softmax support offset and step
### What changes were proposed in this pull request?
use newly impled softmax function in NB

### Why are the changes needed?
to simplify impl

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

### How was this patch tested?
existing testsuite

Closes #32927 from zhengruifeng/softmax__followup.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: Huaxin Gao <huaxin_gao@apple.com>
2021-06-17 22:46:36 -07:00
HyukjinKwon 41af409b7b [SPARK-35303][PYTHON] Enable pinned thread mode by default
### What changes were proposed in this pull request?

PySpark added pinned thread mode at https://github.com/apache/spark/pull/24898 to sync Python thread to JVM thread. Previously, one JVM thread could be reused which ends up with messed inheritance hierarchy such as thread local especially when multiple jobs run in parallel. To completely fix this, we should enable this mode by default.

### Why are the changes needed?

To correctly support parallel job submission and management.

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

Yes, now Python thread is mapped to JVM thread one to one.

### How was this patch tested?

Existing tests should cover it.

Closes #32429 from HyukjinKwon/SPARK-35303.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-18 12:02:29 +09:00
William Hyun 4373b67a6b [MINOR] Add GitHub Action build status badge to the README
### What changes were proposed in this pull request?
This PR aims to add GitHub Action build status badge to README.md.

### Why are the changes needed?
This will improve the visibility of the build status.
- https://github.com/williamhyun/spark/tree/badge#apache-spark

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

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

Closes #32954 from williamhyun/badge.

Authored-by: William Hyun <william@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-06-17 15:25:24 -07:00
Kousuke Saruta 45b7f76295 [SPARK-35095][SS][TESTS] Use ANSI intervals in streaming join tests
### What changes were proposed in this pull request?

This PR extends the following tests to use day-time intervals.

* StreamingOuterJoinSuite.right outer with watermark range condition
* StreamingOuterJoinSuite.left outer with watermark range condition

### Why are the changes needed?

Currently, there are no tests to use day-time intervals.

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

No.

### How was this patch tested?

New assertions.

Closes #32953 from sarutak/stream-join-interval.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-17 22:48:18 +03:00
David Christle 7fcb127674 [SPARK-35670][BUILD] Upgrade ZSTD-JNI to 1.5.0-2
### What changes were proposed in this pull request?
This PR aims to upgrade `zstd-jni` to 1.5.0-2, which uses `zstd` version 1.5.0.

### Why are the changes needed?
Major improvements to Zstd support are targeted for the upcoming 3.2.0 release of Spark. Zstd 1.5.0 introduces significant compression (+25% to 140%) and decompression (~15%) speed improvements in benchmarks described in more detail on the releases page:

- https://github.com/facebook/zstd/releases/tag/v1.5.0

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

### How was this patch tested?
Build passes build tests, but the benchmark tests seem flaky. I am unsure if this change is responsible. The error is:
```
Running org.apache.spark.rdd.CoalescedRDDBenchmark:
21/06/08 18:53:10 ERROR SparkContext: Failed to add file:/home/runner/work/spark/spark/./core/target/scala-2.12/spark-core_2.12-3.2.0-SNAPSHOT-tests.jar to Spark environment
java.lang.IllegalArgumentException: requirement failed: File spark-core_2.12-3.2.0-SNAPSHOT-tests.jar was already registered with a different path (old path = /home/runner/work/spark/spark/core/target/scala-2.12/spark-core_2.12-3.2.0-SNAPSHOT-tests.jar, new path = /home/runner/work/spark/spark/./core/target/scala-2.12/spark-core_2.12-3.2.0-SNAPSHOT-tests.jar
```

https://github.com/dchristle/spark/runs/2776123749?check_suite_focus=true

cc: dongjoon-hyun

Closes #32826 from dchristle/ZSTD150.

Lead-authored-by: David Christle <dchristle@squareup.com>
Co-authored-by: David Christle <dchristle@users.noreply.github.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-17 11:06:50 -07:00
Gengliang Wang 05e2b76852 [SPARK-35720][SQL] Support casting of String to timestamp without time zone type
### What changes were proposed in this pull request?

Extend the Cast expression and support StringType in casting to TimestampWithoutTZType.

Closes #32898

### Why are the changes needed?

To conform the ANSI SQL standard which requires to support such casting.

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

No, the new timestamp type is not released yet.

### How was this patch tested?

Unit test

Closes #32936 from gengliangwang/castStringToTswtz.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-18 02:02:10 +08:00
Angerszhuuuu 79362c4efc [SPARK-34898][CORE] We should log SparkListenerExecutorMetricsUpdateEvent of driver appropriately when spark.eventLog.logStageExecutorMetrics is true
### What changes were proposed in this pull request?
In current EventLoggingListener, we won't write SparkListenerExecutorMetricsUpdate message to event log file at all

```
override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = {
  if (shouldLogStageExecutorMetrics) {
    event.executorUpdates.foreach { case (stageKey1, newPeaks) =>
      liveStageExecutorMetrics.foreach { case (stageKey2, metricsPerExecutor) =>
        // If the update came from the driver, stageKey1 will be the dummy key (-1, -1),
        // so record those peaks for all active stages.
        // Otherwise, record the peaks for the matching stage.
        if (stageKey1 == DRIVER_STAGE_KEY || stageKey1 == stageKey2) {
          val metrics = metricsPerExecutor.getOrElseUpdate(
            event.execId, new ExecutorMetrics())
          metrics.compareAndUpdatePeakValues(newPeaks)
        }
      }
    }
  }
}
```

In history server's restful API about executor, we can get Executor's metrics but can't get all driver's metrics. Executor's executor metrics can be updated with TaskEnd event etc...

So in this pr, I add support to log SparkListenerExecutorMetricsUpdateEvent of `driver` when `spark.eventLog.logStageExecutorMetrics` is true.

### Why are the changes needed?
Make user can got driver's peakMemoryMetrics in SHS.

### Does this PR introduce _any_ user-facing change?
 user can got driver's executor metrics in SHS's restful API.

### How was this patch tested?
Mannul test

Closes #31992 from AngersZhuuuu/SPARK-34898.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-06-17 12:08:10 -05:00
allisonwang-db 0d900b6cfa [SPARK-35789][SQL] Refine lateral join syntax to only allow subqueries
### What changes were proposed in this pull request?
This PR is a follow-up for SPARK-34382. It refines the lateral join syntax to only allow the LATERAL keyword to be in front of subqueries, instead of all `relationPriamry`. For example, `SELECT * FROM t1, LATERAL t2` should not be allowed.

### Why are the changes needed?
To be consistent with Postgres.

### Does this PR introduce _any_ user-facing change?
Yes. After this PR, the LATERAL keyword can only be in front of subqueries.

```scala
sql("SELECT * FROM t1, LATERAL t2")

org.apache.spark.sql.catalyst.parser.ParseException:
LATERAL can only be used with subquery(line 1, pos 26)

== SQL ==
select * from t1, lateral t2
--------------------------^^^
```

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

Closes #32937 from allisonwang-db/spark-35789-lateral-join-parser.

Authored-by: allisonwang-db <allison.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-17 16:47:30 +00:00
yi.wu 509c076bc0 [SPARK-34054][CORE] BlockManagerDecommissioner code cleanup
### What changes were proposed in this pull request?

This PR cleans up the code of `BlockManagerDecommissioner`. It includes a few changes:

* Only create `BlockManagerDecommissioner` instance when shuffle or RDD blocks requires migration:
   there's no need to create `BlockManagerDecommissioner` instance if only `STORAGE_DECOMMISSION_ENABLED=true` and to check blocks migration in `shutdownThread`.

* Shut down the migration thread more gracefully:

  1. we'd better not log errors if the `BlockManagerDecommissioner.stop()` is invoked explicitly. But currently, users will see
    <details>

      <summary>error message</summary>

    ```
    21/01/04 20:11:52 ERROR BlockManagerDecommissioner: Error while waiting for block to migrate
    java.lang.InterruptedException: sleep interrupted
	    at java.lang.Thread.sleep(Native Method)
	    at org.apache.spark.storage.BlockManagerDecommissioner$ShuffleMigrationRunnable.run(BlockManagerDecommissioner.scala:83)
	    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	    at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	    at org.apache.spark.util.threads.SparkThreadLocalCapturingRunnable.$anonfun$run$1(SparkThreadLocalForwardingThreadPoolExecutor.scala:104)
	    at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	    at org.apache.spark.util.threads.SparkThreadLocalCapturingHelper.runWithCaptured(SparkThreadLocalForwardingThreadPoolExecutor.scala:68)
	    at org.apache.spark.util.threads.SparkThreadLocalCapturingHelper.runWithCaptured$(SparkThreadLocalForwardingThreadPoolExecutor.scala:54)
	    at org.apache.spark.util.threads.SparkThreadLocalCapturingRunnable.runWithCaptured(SparkThreadLocalForwardingThreadPoolExecutor.scala:101)
	    at org.apache.spark.util.threads.SparkThreadLocalCapturingRunnable.run(SparkThreadLocalForwardingThreadPoolExecutor.scala:104)
	    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)
    ```
    </details>

   2. don't shut down a thread pool like below since `shutdown()` actually doesn't block to wait for running tasks finish:

      ```scala
      executor.shutdown()
      executor.shutdownNow()
      ```

* Avoid initiating `shuffleMigrationPool` when it's unnecessary:
 Currently, it's always initiated even if shuffle block migration is disabled. (`BlockManagerDecommissioner.stop()` -> `stopOffloadingShuffleBlocks()` -> initiate `shuffleMigrationPool`)

* Unify the terminologies between `offload` and `migrate`:
  replace `offload` with `migrate`

* Do not add back the shuffle blocks when it exceeds the max failure number:
   this avoids unnecessary operations

* Do not try `decommissionRddCacheBlocks()` if we already know there are no available peers

* Clean up logs:
   Currently, we have many different description for the same thing, which is not good for the user experience

* Other cleanups

### Why are the changes needed?

code clean up

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

Yes, users will not see misleading logs, e.g., the interrupted error.

### How was this patch tested?

Update a unite test since we change the behavior of creating the `BlockManagerDecommissioner` instance.

Other changes are only code cleanup so they won't cause behaviour change. So passing the existing tests should be enough.

Closes #31102 from Ngone51/stop-decommission-gracefully.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-17 15:00:18 +00:00
gengjiaan ee2d8ae322 [SPARK-35378][SQL][FOLLOWUP] Move CommandResult to catalyst.plans.logical
### What changes were proposed in this pull request?
https://github.com/apache/spark/pull/32513 added the case class `CommandResult` in package `org.apache.spark.sql.expression`. It is not suitable, so this PR move `CommandResult` from `org.apache.spark.sql.expression` to `org.apache.spark.sql.catalyst.plans.logical`.

### Why are the changes needed?
Make `CommandResult` in suitable package.

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

### How was this patch tested?
No need.

Closes #32942 from beliefer/SPARK-35378-followup.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-17 07:47:38 -07:00
Peter Toth abf9675a75 [SPARK-35798][SQL] Fix SparkPlan.sqlContext usage
### What changes were proposed in this pull request?
There might be `SparkPlan` nodes where canonicalization on executor side can cause issues. This is a follow-up fix to conversation https://github.com/apache/spark/pull/32885/files#r651019687.

### Why are the changes needed?
To avoid potential NPEs when canonicalization happens on executors.

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

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

Closes #32947 from peter-toth/SPARK-35798-fix-sparkplan.sqlcontext-usage.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-17 13:49:38 +00:00
Linhong Liu b86a69f026 [SPARK-35792][SQL] View should not capture configs used in RelationConversions
### What changes were proposed in this pull request?
`RelationConversions` is actually an optimization rule while it's executed in the analysis phase.
For view, it's designed to only capture semantic configs, so we should ignore the optimization
configs that will be used in the analysis phase.

This PR also fixes the issue that view resolution will always use the default value for uncaptured config

### Why are the changes needed?
Bugfix

### Does this PR introduce _any_ user-facing change?
Yes, after this PR view resolution will respect the values set in the current session for the below configs
```
"spark.sql.hive.convertMetastoreParquet"
"spark.sql.hive.convertMetastoreOrc"
"spark.sql.hive.convertInsertingPartitionedTable"
"spark.sql.hive.convertMetastoreCtas"
```

### How was this patch tested?
By running new UT:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *HiveSQLViewSuite"
```

Closes #32941 from linhongliu-db/SPARK-35792-ignore-convert-configs.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-17 21:40:53 +08:00
Angerszhuuuu 234163fbe0 [SPARK-35732][SQL] Parse DayTimeIntervalType from JSON
### What changes were proposed in this pull request?
Support Parse DayTimeIntervalType from JSON

### Why are the changes needed?
this will allow to store day-second intervals as table columns into Hive external catalog.

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

### How was this patch tested?
Added UT

Closes #32930 from AngersZhuuuu/SPARK-35732.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-17 12:54:34 +03:00
Wenchen Fan 0c5a01a78c [SPARK-35378][SQL][FOLLOWUP] Restore the command execution name for DataFrameWriterV2
### What changes were proposed in this pull request?

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

It's hard to keep the command execution name for `DataFrameWriter`, as the command logical plan is a bit messy (DS v1, file source and hive and different command logical plans) and sometimes it's hard to distinguish "insert" and "save".

However, `DataFrameWriterV2` only produce v2 commands which are pretty clean. It's easy to keep the command execution name for them.

### Why are the changes needed?

less breaking changes.

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

no

### How was this patch tested?

N/A

Closes #32919 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-17 08:55:42 +00:00
copperybean 939ae91e00 [SPARK-35130][SQL] Add make_dt_interval function to construct DayTimeIntervalType value
### What changes were proposed in this pull request?
Providing a new function make_dt_interval to construct DayTimeIntervalType value

### Why are the changes needed?
As the JIRA described, we should provide a function to construct DayTimeIntervalType value

### Does this PR introduce _any_ user-facing change?
Yes, a new make_dt_interval function provided

### How was this patch tested?
Updated UTs, manual testing

Closes #32601 from copperybean/work.

Authored-by: copperybean <copperybean.zhang@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-17 10:01:16 +03:00
Angerszhuuuu 0e554d44df [SPARK-35770][SQL] Parse YearMonthIntervalType from JSON
### What changes were proposed in this pull request?
Parse YearMonthIntervalType from JSON.

### Why are the changes needed?
This will allow to store year-month intervals as table columns into Hive external catalog.

### Does this PR introduce _any_ user-facing change?
People can store year-month interval types as json string.

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

Closes #32929 from AngersZhuuuu/SPARK-35770.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-17 09:51:47 +03:00
kudhru 8aeed08d04 [SPARK-35757][CORE] Add bitwise AND operation and functionality for intersecting bloom filters
### What changes were proposed in this pull request?
This change is for [SPARK-35757](https://issues.apache.org/jira/browse/SPARK-35757) and does the following:

1. adds bitwise AND operation to BitArray (similar to existing `putAll` method)
2. adds an intersect operation for combining bloom filters using bitwise AND operation (similar to existing `mergeInPlace` method).

### Why are the changes needed?
The current bloom filter library only allows combining two bloom filters using OR operation. It is useful to have AND operation as well.

### Does this PR introduce _any_ user-facing change?
No, just adds new methods.

### How was this patch tested?
Just the existing tests.

Closes #32907 from kudhru/master.

Lead-authored-by: kudhru <gargdhruv36@gmail.com>
Co-authored-by: Dhruv Kumar <kudhru@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-17 06:29:33 +00:00
Cheng Su e0d81d9b71 [SPARK-35791][SQL] Release on-going map properly for NULL-aware ANTI join
### What changes were proposed in this pull request?

NULL-aware ANTI join (https://issues.apache.org/jira/browse/SPARK-32290) detects NULL join keys during building the map for `HashedRelation`, and will immediately return `HashedRelationWithAllNullKeys` without taking care of the map built already. Before returning `HashedRelationWithAllNullKeys`, the map needs to be freed properly to save memory and keep memory accounting correctly.

### Why are the changes needed?

Save memory and keep memory accounting correctly for the join query.

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

No.

### How was this patch tested?

Existing unit tests introduced in https://github.com/apache/spark/pull/29104 .

Closes #32939 from c21/free-null-aware.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-17 13:57:35 +08:00