Commit graph

31072 commits

Author SHA1 Message Date
Yang He f093f3b939 [SPARK-36780][BUILD] Make dev/mima runs on Java 17
### What changes were proposed in this pull request?

Java 17 has been officially released. This PR makes `dev/mima` runs on Java 17.

### Why are the changes needed?

To make tests pass on Java 17.

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

No.

### How was this patch tested?

Manual test.

Closes #34022 from RabbidHY/SPARK-36780.

Lead-authored-by: Yang He <stitch106hy@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 5d0889bf36)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-17 08:54:57 -07:00
Angerszhuuuu 61d7f1da1b [SPARK-36767][SQL] ArrayMin/ArrayMax/SortArray/ArraySort add comment and Unit test
### What changes were proposed in this pull request?
Add comment about how ArrayMin/ArrayMax/SortArray/ArraySort handle NaN and add Unit test for this

### Why are the changes needed?
Add Unit test

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

### How was this patch tested?
Added UT

Closes #34008 from AngersZhuuuu/SPARK-36740.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 69e006dd53)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-17 21:42:21 +08:00
Liang-Chi Hsieh 895218996a [SPARK-36673][SQL] Fix incorrect schema of nested types of union
### What changes were proposed in this pull request?

This patch proposes to fix incorrect schema of `union`.

### Why are the changes needed?

The current `union` result of nested struct columns is incorrect. By definition of `union` API, it should resolve columns by position, not by name. Right now when determining the `output` (aka. the schema) of union plan, we use `merge` API which actually merges two structs (simply think it as concatenate fields from two structs if not overlapping). The merging behavior doesn't match the `union` definition.

So currently we get incorrect schema but the query result is correct. We should fix the incorrect schema.

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

Yes, fixing a bug of incorrect schema.

### How was this patch tested?

Added unit test.

Closes #34025 from viirya/SPARK-36673.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit cdd7ae937d)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-17 21:37:40 +08:00
Jungtaek Lim af7dd18a5e [SPARK-36764][SS][TEST] Fix race-condition on "ensure continuous stream is being used" in KafkaContinuousTest
### What changes were proposed in this pull request?

The test “ensure continuous stream is being used“ in KafkaContinuousTest quickly checks the actual type of the execution, and stops the query. Stopping the streaming query in continuous mode is done by interrupting query execution thread and join with indefinite timeout.

In parallel, started streaming query is going to generate execution plan, including running optimizer. Some parts of SessionState can be built at that time, as they are defined as lazy. The problem is, some of them seem to “swallow” the InterruptedException and let the thread run continuously.

That said, the query can’t indicate whether there is a request on stopping query, so the query won’t stop.

This PR fixes such scenario via ensuring that streaming query has started before the test stops the query.

### Why are the changes needed?

Race-condition could end up with test hang till test framework marks it as timed-out.

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

No.

### How was this patch tested?

Existing tests.

Closes #34004 from HeartSaVioR/SPARK-36764.

Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 6099edc66e)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-17 21:28:19 +08:00
Angerszhuuuu a78c4c44ed [SPARK-36741][SQL] ArrayDistinct handle duplicated Double.NaN and Float.Nan
### What changes were proposed in this pull request?
For query
```
select array_distinct(array(cast('nan' as double), cast('nan' as double)))
```
This returns [NaN, NaN], but it should return [NaN].
This issue is caused by `OpenHashSet` can't handle `Double.NaN` and `Float.NaN` too.
In this pr fix this based on https://github.com/apache/spark/pull/33955

### Why are the changes needed?
Fix bug

### Does this PR introduce _any_ user-facing change?
ArrayDistinct won't show duplicated `NaN` value

### How was this patch tested?
Added UT

Closes #33993 from AngersZhuuuu/SPARK-36741.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit e356f6aa11)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-17 20:48:39 +08:00
Wenchen Fan 16215755b7 [SPARK-36789][SQL] Use the correct constant type as the null value holder in array functions
### What changes were proposed in this pull request?

In array functions, we use constant 0 as the placeholder when adding a null value to an array buffer. This PR makes sure the constant 0 matches the type of the array element.

### Why are the changes needed?

Fix a potential bug. Somehow we can hit this bug sometimes after https://github.com/apache/spark/pull/33955 .

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

No

### How was this patch tested?

existing tests

Closes #34029 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 4145498826)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-17 16:50:01 +09:00
Hyukjin Kwon 7d7c9915bb [SPARK-36788][SQL] Change log level of AQE for non-supported plans from warning to debug
### What changes were proposed in this pull request?

This PR suppresses the warnings for plans where AQE is not supported. Currently we show the warnings such as:

```
org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: Sort [a#324881 DESC NULLS FIRST], true, 23
+- Scan ExistingRDD[a#324881]
```

for every plan that AQE is not supported.

### Why are the changes needed?

It's too noisy now. Below is the example of `SortSuite` run:

```
14:51:40.675 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: Sort [a#324881 DESC NULLS FIRST], true, 23
+- Scan ExistingRDD[a#324881]
.
[info] - sorting on DayTimeIntervalType(0,1) with nullable=true, sortOrder=List('a DESC NULLS FIRST) (785 milliseconds)
14:51:41.416 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: ReferenceSort [a#324884 ASC NULLS FIRST], true
+- Scan ExistingRDD[a#324884]
.
14:51:41.467 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: Sort [a#324884 ASC NULLS FIRST], true, 23
+- Scan ExistingRDD[a#324884]
.
[info] - sorting on DayTimeIntervalType(0,1) with nullable=false, sortOrder=List('a ASC NULLS FIRST) (796 milliseconds)
14:51:42.210 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: ReferenceSort [a#324887 ASC NULLS LAST], true
+- Scan ExistingRDD[a#324887]
.
14:51:42.259 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: Sort [a#324887 ASC NULLS LAST], true, 23
+- Scan ExistingRDD[a#324887]
.
[info] - sorting on DayTimeIntervalType(0,1) with nullable=false, sortOrder=List('a ASC NULLS LAST) (797 milliseconds)
14:51:43.009 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: ReferenceSort [a#324890 DESC NULLS LAST], true
+- Scan ExistingRDD[a#324890]
.
14:51:43.061 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: Sort [a#324890 DESC NULLS LAST], true, 23
+- Scan ExistingRDD[a#324890]
.
[info] - sorting on DayTimeIntervalType(0,1) with nullable=false, sortOrder=List('a DESC NULLS LAST) (848 milliseconds)
14:51:43.857 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: ReferenceSort [a#324893 DESC NULLS FIRST], true
+- Scan ExistingRDD[a#324893]
.
14:51:43.903 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: Sort [a#324893 DESC NULLS FIRST], true, 23
+- Scan ExistingRDD[a#324893]
.
[info] - sorting on DayTimeIntervalType(0,1) with nullable=false, sortOrder=List('a DESC NULLS FIRST) (827 milliseconds)
14:51:44.682 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: ReferenceSort [a#324896 ASC NULLS FIRST], true
+- Scan ExistingRDD[a#324896]
.
14:51:44.748 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: Sort [a#324896 ASC NULLS FIRST], true, 23
+- Scan ExistingRDD[a#324896]
.
[info] - sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a ASC NULLS FIRST) (565 milliseconds)
14:51:45.248 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: ReferenceSort [a#324899 ASC NULLS LAST], true
+- Scan ExistingRDD[a#324899]
.
14:51:45.312 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: Sort [a#324899 ASC NULLS LAST], true, 23
+- Scan ExistingRDD[a#324899]
.
[info] - sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a ASC NULLS LAST) (591 milliseconds)
14:51:45.841 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: ReferenceSort [a#324902 DESC NULLS LAST], true
+- Scan ExistingRDD[a#324902]
.
14:51:45.905 WARN org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan: spark.sql.adaptive.enabled is enabled but is not supported for query: Sort [a#324902 DESC NULLS LAST], true, 23
+- Scan ExistingRDD[a#324902]
.
```

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

Yes, it will show less warnings to users. Note that AQE is enabled by default from Spark 3.2, see SPARK-33679

### How was this patch tested?

Manually tested via unittests.

Closes #34026 from HyukjinKwon/minor-log-level.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 917d7dad4d)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-17 12:01:53 +09:00
Wenchen Fan c1bfe1a5c4 [SPARK-36783][SQL] ScanOperation should not push Filter through nondeterministic Project
### What changes were proposed in this pull request?

`ScanOperation` collects adjacent Projects and Filters. The caller side always assume that the collected Filters should run before collected Projects, which means `ScanOperation` effectively pushes Filter through Project.

Following `PushPredicateThroughNonJoin`, we should not push Filter through nondeterministic Project. This PR fixes `ScanOperation` to follow this rule.

### Why are the changes needed?

Fix a bug that violates the semantic of nondeterministic expressions.

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

Most likely no change, but in some cases, this is a correctness bug fix which changes the query result.

### How was this patch tested?

existing tests

Closes #34023 from cloud-fan/scan.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit dfd5237c0c)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-17 10:51:29 +08:00
Josh Rosen 3502fda783 [SPARK-36774][CORE][TESTS] Move SparkSubmitTestUtils to core module and use it in SparkSubmitSuite
### What changes were proposed in this pull request?

This PR refactors test code in order to improve the debugability of `SparkSubmitSuite`.

The `sql/hive` module contains a `SparkSubmitTestUtils` helper class which launches `spark-submit` and captures its output in order to display better error messages when tests fail. This helper is currently used by `HiveSparkSubmitSuite` and `HiveExternalCatalogVersionsSuite`, but isn't used by `SparkSubmitSuite`.

In this PR, I moved `SparkSubmitTestUtils` and `ProcessTestUtils` into the `core` module and updated `SparkSubmitSuite`, `BufferHolderSparkSubmitSuite`, and `WholestageCodegenSparkSubmitSuite` to use the relocated helper classes. This required me to change `SparkSubmitTestUtils` to make its timeouts configurable and to generalize its method for locating the `spark-submit` binary.

### Why are the changes needed?

Previously, `SparkSubmitSuite` tests would fail with messages like:

```
[info] - launch simple application with spark-submit *** FAILED *** (1 second, 832 milliseconds)
[info]   Process returned with exit code 101. See the log4j logs for more detail. (SparkSubmitSuite.scala:1551)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
```

which require the Spark developer to hunt in log4j logs in order to view the logs from the failed `spark-submit` command.

After this change, those tests will fail with detailed error messages that include the text of failed command plus timestamped logs captured from the failed proces:

```
[info] - launch simple application with spark-submit *** FAILED *** (2 seconds, 800 milliseconds)
[info]   spark-submit returned with exit code 101.
[info]   Command line: '/Users/joshrosen/oss-spark/bin/spark-submit' '--class' 'invalidClassName' '--name' 'testApp' '--master' 'local' '--conf' 'spark.ui.enabled=false' '--conf' 'spark.master.rest.enabled=false' 'file:/Users/joshrosen/oss-spark/target/tmp/spark-0a8a0c93-3aaf-435d-9cf3-b97abd318d91/testJar-1631768004882.jar'
[info]
[info]   2021-09-15 21:53:26.041 - stderr> SLF4J: Class path contains multiple SLF4J bindings.
[info]   2021-09-15 21:53:26.042 - stderr> SLF4J: Found binding in [jar:file:/Users/joshrosen/oss-spark/assembly/target/scala-2.12/jars/slf4j-log4j12-1.7.30.jar!/org/slf4j/impl/StaticLoggerBinder.class]
[info]   2021-09-15 21:53:26.042 - stderr> SLF4J: Found binding in [jar:file:/Users/joshrosen/.m2/repository/org/slf4j/slf4j-log4j12/1.7.30/slf4j-log4j12-1.7.30.jar!/org/slf4j/impl/StaticLoggerBinder.class]
[info]   2021-09-15 21:53:26.042 - stderr> SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
[info]   2021-09-15 21:53:26.042 - stderr> SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory]
[info]   2021-09-15 21:53:26.619 - stderr> Error: Failed to load class invalidClassName. (SparkSubmitTestUtils.scala:97)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
```

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

No.

### How was this patch tested?

I manually ran the affected test suites.

Closes #34013 from JoshRosen/SPARK-36774-move-SparkSubmitTestUtils-to-core.

Authored-by: Josh Rosen <joshrosen@databricks.com>
Signed-off-by: Josh Rosen <joshrosen@databricks.com>
(cherry picked from commit 3ae6e6775b)
Signed-off-by: Josh Rosen <joshrosen@databricks.com>
2021-09-16 14:31:22 -07:00
Dongjoon Hyun fbd24621ce [SPARK-36759][BUILD][FOLLOWUP] Update version in scala-2.12 profile and doc
### What changes were proposed in this pull request?

This is a follow-up to fix the leftover during switching the Scala version.

### Why are the changes needed?

This should be consistent.

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

No.

### How was this patch tested?

This is not tested by UT. We need to check manually. There is no more `2.12.14`.
```
$ git grep 2.12.14
R/pkg/tests/fulltests/test_sparkSQL.R:               c(as.Date("2012-12-14"), as.Date("2013-12-15"), as.Date("2014-12-16")))
data/mllib/ridge-data/lpsa.data:3.5307626,0.987291634724086 -0.36279314978779 -0.922212414640967 0.232904453212813 -0.522940888712441 1.79270085261407 0.342627053981254 1.26288870310799
sql/hive/src/test/resources/data/files/over10k:-3|454|65705|4294967468|62.12|14.32|true|mike white|2013-03-01 09:11:58.703087|40.18|joggying
```

Closes #34020 from dongjoon-hyun/SPARK-36759-2.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit adbea252db)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-16 05:11:05 -07:00
Kousuke Saruta dda43fe5ee [SPARK-36777][INFRA] Move Java 17 on GitHub Actions from EA to LTS release
### What changes were proposed in this pull request?

This PR aims to move Java 17 on GA from early access release to LTS release.

### Why are the changes needed?

Java 17 LTS was released a few days ago and it's available on GA.

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

No.

### How was this patch tested?

GA itself.

Closes #34017 from sarutak/ga-java17.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
(cherry picked from commit 89a9456b13)
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-09-16 18:05:04 +08:00
Gengliang Wang d20ed030a8 [SPARK-36775][DOCS] Add documentation for ANSI store assignment rules
### What changes were proposed in this pull request?

Add documentation for ANSI store assignment rules for
- the valid source/target type combinations
- runtime error will happen on numberic overflow

### Why are the changes needed?

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

No

### How was this patch tested?

Build docs and preview:
![image](https://user-images.githubusercontent.com/1097932/133554600-8c80c0a9-8753-4c01-94d0-994d8082e319.png)

Closes #34014 from gengliangwang/addStoreAssignDoc.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit ff7705ad2a)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-16 15:50:57 +08:00
Dongjoon Hyun 6352846f1c [SPARK-36732][BUILD][FOLLOWUP] Fix dependency manifest 2021-09-15 23:38:48 -07:00
Dongjoon Hyun 63b8417794 [SPARK-36732][SQL][BUILD] Upgrade ORC to 1.6.11
### What changes were proposed in this pull request?

This PR aims to upgrade Apache ORC to 1.6.11 to bring the latest bug fixes.

### Why are the changes needed?

Apache ORC 1.6.11 has the following fixes.
- https://issues.apache.org/jira/projects/ORC/versions/12350499

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

No.

### How was this patch tested?

Pass the CIs.

Closes #33971 from dongjoon-hyun/SPARK-36732.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit c217797297)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-15 23:36:36 -07:00
Dongjoon Hyun 2067661869 [SPARK-36759][BUILD] Upgrade Scala to 2.12.15
### What changes were proposed in this pull request?

This PR aims to upgrade Scala to 2.12.15 to support Java 17/18 better.

### Why are the changes needed?

Scala 2.12.15 improves compatibility with JDK 17 and 18:

https://github.com/scala/scala/releases/tag/v2.12.15

- Avoids IllegalArgumentException in JDK 17+ for lambda deserialization
- Upgrades to ASM 9.2, for JDK 18 support in optimizer

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

Yes, this is a Scala version change.

### How was this patch tested?

Pass the CIs

Closes #33999 from dongjoon-hyun/SPARK-36759.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 16f1f71ba5)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-15 13:43:36 -07:00
Chao Sun a7dc8242ea [SPARK-36726] Upgrade Parquet to 1.12.1
### What changes were proposed in this pull request?

Upgrade Apache Parquet to 1.12.1

### Why are the changes needed?

Parquet 1.12.1 contains the following bug fixes:
- PARQUET-2064: Make Range public accessible in RowRanges
- PARQUET-2022: ZstdDecompressorStream should close `zstdInputStream`
- PARQUET-2052: Integer overflow when writing huge binary using dictionary encoding
- PARQUET-1633: Fix integer overflow
- PARQUET-2054: fix TCP leaking when calling ParquetFileWriter.appendFile
- PARQUET-2072: Do Not Determine Both Min/Max for Binary Stats
- PARQUET-2073: Fix estimate remaining row count in ColumnWriteStoreBase
- PARQUET-2078: Failed to read parquet file after writing with the same

In particular PARQUET-2078 is a blocker for the upcoming Apache Spark 3.2.0 release.

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

No

### How was this patch tested?

Existing tests + a new test for the issue in SPARK-36696

Closes #33969 from sunchao/upgrade-parquet-12.1.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
(cherry picked from commit a927b0836b)
Signed-off-by: DB Tsai <d_tsai@apple.com>
2021-09-15 19:17:49 +00:00
dgd-contributor 017bce7b11 [SPARK-36722][PYTHON] Fix Series.update with another in same frame
### What changes were proposed in this pull request?
Fix Series.update with another in same frame

also add test for update series in diff frame

### Why are the changes needed?
Fix Series.update with another in same frame

Pandas behavior:
``` python
>>> pdf = pd.DataFrame(
...     {"a": [None, 2, 3, 4, 5, 6, 7, 8, None], "b": [None, 5, None, 3, 2, 1, None, 0, 0]},
... )
>>> pdf
     a    b
0  NaN  NaN
1  2.0  5.0
2  3.0  NaN
3  4.0  3.0
4  5.0  2.0
5  6.0  1.0
6  7.0  NaN
7  8.0  0.0
8  NaN  0.0
>>> pdf.a.update(pdf.b)
>>> pdf
     a    b
0  NaN  NaN
1  5.0  5.0
2  3.0  NaN
3  3.0  3.0
4  2.0  2.0
5  1.0  1.0
6  7.0  NaN
7  0.0  0.0
8  0.0  0.0
```

### Does this PR introduce _any_ user-facing change?
Before
```python
>>> psdf = ps.DataFrame(
...     {"a": [None, 2, 3, 4, 5, 6, 7, 8, None], "b": [None, 5, None, 3, 2, 1, None, 0, 0]},
... )

>>> psdf.a.update(psdf.b)
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/Users/dgd/spark/python/pyspark/pandas/series.py", line 4551, in update
    combined = combine_frames(self._psdf, other._psdf, how="leftouter")
  File "/Users/dgd/spark/python/pyspark/pandas/utils.py", line 141, in combine_frames
    assert not same_anchor(
AssertionError: We don't need to combine. `this` and `that` are same.
>>>
```

After
```python
>>> psdf = ps.DataFrame(
...     {"a": [None, 2, 3, 4, 5, 6, 7, 8, None], "b": [None, 5, None, 3, 2, 1, None, 0, 0]},
... )

>>> psdf.a.update(psdf.b)
>>> psdf
     a    b
0  NaN  NaN
1  5.0  5.0
2  3.0  NaN
3  3.0  3.0
4  2.0  2.0
5  1.0  1.0
6  7.0  NaN
7  0.0  0.0
8  0.0  0.0
>>>
```

### How was this patch tested?
unit tests

Closes #33968 from dgd-contributor/SPARK-36722_fix_update_same_anchor.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
(cherry picked from commit c15072cc73)
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-09-15 11:08:12 -07:00
Angerszhuuuu 75bffd972d [SPARK-36755][SQL] ArraysOverlap should handle duplicated Double.NaN and Float.NaN
### What changes were proposed in this pull request?
For query
```
select arrays_overlap(array(cast('nan' as double), 1d), array(cast('nan' as double)))
```
This returns [false], but it should return [true].
This issue is caused by `scala.mutable.HashSet` can't handle `Double.NaN` and `Float.NaN`.

### Why are the changes needed?
Fix bug

### Does this PR introduce _any_ user-facing change?
arrays_overlap won't handle equal `NaN` value

### How was this patch tested?
Added UT

Closes #34006 from AngersZhuuuu/SPARK-36755.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit b665782f0d)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-15 22:32:18 +08:00
Angerszhuuuu e64155691f [SPARK-36702][SQL][FOLLOWUP] ArrayUnion handle duplicated Double.NaN and Float.NaN
### What changes were proposed in this pull request?
According to https://github.com/apache/spark/pull/33955#discussion_r708570515 use normalized  NaN

### Why are the changes needed?
Use normalized NaN for duplicated NaN value

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

### How was this patch tested?
Exiting UT

Closes #34003 from AngersZhuuuu/SPARK-36702-FOLLOWUP.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 638085953f)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-15 22:04:24 +08:00
Dongjoon Hyun 6a1dacb6b6 [SPARK-36712][BUILD][FOLLOWUP] Improve the regex to avoid breaking pom.xml
### What changes were proposed in this pull request?

This PR aims to fix the regex to avoid breaking `pom.xml`.

### Why are the changes needed?

**BEFORE**
```
$ dev/change-scala-version.sh 2.12
$ git diff | head -n10
diff --git a/core/pom.xml b/core/pom.xml
index dbde22f2bf..6ed368353b 100644
--- a/core/pom.xml
+++ b/core/pom.xml
 -35,7 +35,7
   </properties>

   <dependencies>
-    <!--<!--
```

**AFTER**
Since the default Scala version is `2.12`, the following `no-op` is the correct behavior which is consistent with the previous behavior.
```
$ dev/change-scala-version.sh 2.12
$ git diff
```

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

No. This is a dev only change.

### How was this patch tested?

Manually.

Closes #33996 from dongjoon-hyun/SPARK-36712.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit d730ef24fe)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-14 16:27:02 -07:00
Angerszhuuuu a472612eb8 [SPARK-36702][SQL] ArrayUnion handle duplicated Double.NaN and Float.Nan
### What changes were proposed in this pull request?
For query
```
select array_union(array(cast('nan' as double), cast('nan' as double)), array())
```
This returns [NaN, NaN], but it should return [NaN].
This issue is caused by `OpenHashSet` can't handle `Double.NaN` and `Float.NaN` too.
In this pr we add a wrap for OpenHashSet that can handle `null`, `Double.NaN`, `Float.NaN` together

### Why are the changes needed?
Fix bug

### Does this PR introduce _any_ user-facing change?
ArrayUnion won't show duplicated `NaN` value

### How was this patch tested?
Added UT

Closes #33955 from AngersZhuuuu/SPARK-36702-WrapOpenHashSet.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit f71f37755d)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-14 18:26:02 +08:00
Minchu Yang 4a486f40cf [SPARK-36705][FOLLOW-UP] Fix unnecessary logWarning when PUSH_BASED_SHUFFLE_ENABLED is set to false
### What changes were proposed in this pull request?

Only throw logWarning when `PUSH_BASED_SHUFFLE_ENABLED` is set to true and `canDoPushBasedShuffle` is false

### Why are the changes needed?

Currently, this logWarning will still be printed out even when `PUSH_BASED_SHUFFLE_ENABLED` is set to false, which is unnecessary.

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

No.

### How was this patch tested?

Passed existing UT.

Closes #33984 from rmcyang/SPARK-36705-follow-up.

Authored-by: Minchu Yang <minyang@minyang-mn3.linkedin.biz>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 2d7dc7c7ce)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-09-13 23:24:25 -05:00
Fu Chen 303590b3e9 [SPARK-36715][SQL] InferFiltersFromGenerate should not infer filter for udf
### What changes were proposed in this pull request?

Fix InferFiltersFromGenerate bug, InferFiltersFromGenerate should not infer filter for generate when the children contain an expression which is instance of `org.apache.spark.sql.catalyst.expressions.UserDefinedExpression`.
Before this pr, the following case will throw an exception.

```scala
spark.udf.register("vec", (i: Int) => (0 until i).toArray)
sql("select explode(vec(8)) as c1").show
```

```
Once strategy's idempotence is broken for batch Infer Filters
 GlobalLimit 21                                                        GlobalLimit 21
 +- LocalLimit 21                                                      +- LocalLimit 21
    +- Project [cast(c1#3 as string) AS c1#12]                            +- Project [cast(c1#3 as string) AS c1#12]
       +- Generate explode(vec(8)), false, [c1#3]                            +- Generate explode(vec(8)), false, [c1#3]
          +- Filter ((size(vec(8), true) > 0) AND isnotnull(vec(8)))            +- Filter ((size(vec(8), true) > 0) AND isnotnull(vec(8)))
!            +- OneRowRelation                                                     +- Filter ((size(vec(8), true) > 0) AND isnotnull(vec(8)))
!                                                                                     +- OneRowRelation

java.lang.RuntimeException:
Once strategy's idempotence is broken for batch Infer Filters
 GlobalLimit 21                                                        GlobalLimit 21
 +- LocalLimit 21                                                      +- LocalLimit 21
    +- Project [cast(c1#3 as string) AS c1#12]                            +- Project [cast(c1#3 as string) AS c1#12]
       +- Generate explode(vec(8)), false, [c1#3]                            +- Generate explode(vec(8)), false, [c1#3]
          +- Filter ((size(vec(8), true) > 0) AND isnotnull(vec(8)))            +- Filter ((size(vec(8), true) > 0) AND isnotnull(vec(8)))
!            +- OneRowRelation                                                     +- Filter ((size(vec(8), true) > 0) AND isnotnull(vec(8)))
!                                                                                     +- OneRowRelation

	at org.apache.spark.sql.errors.QueryExecutionErrors$.onceStrategyIdempotenceIsBrokenForBatchError(QueryExecutionErrors.scala:1200)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.checkBatchIdempotence(RuleExecutor.scala:168)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:254)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:200)
	at scala.collection.immutable.List.foreach(List.scala:431)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:200)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:179)
	at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:179)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:138)
	at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:196)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
	at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:196)
	at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:134)
	at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:130)
	at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:148)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$executedPlan$1(QueryExecution.scala:166)
	at org.apache.spark.sql.execution.QueryExecution.withCteMap(QueryExecution.scala:73)
	at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:163)
	at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:163)
	at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:214)
	at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:259)
	at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:228)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:98)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
	at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3731)
	at org.apache.spark.sql.Dataset.head(Dataset.scala:2755)
	at org.apache.spark.sql.Dataset.take(Dataset.scala:2962)
	at org.apache.spark.sql.Dataset.getRows(Dataset.scala:288)
	at org.apache.spark.sql.Dataset.showString(Dataset.scala:327)
	at org.apache.spark.sql.Dataset.show(Dataset.scala:807)
```

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

No, only bug fix.

### How was this patch tested?

Unit test.

Closes #33956 from cfmcgrady/SPARK-36715.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 52c5ff20ca)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-14 09:26:21 +09:00
Leona Yoda b3488a50d7 [SPARK-36739][DOCS][PYTHON] Add apache license headers to makefiles
### What changes were proposed in this pull request?

Add apache license headers to makefiles of PySpark documents.

### Why are the changes needed?

Makefiles of PySpark documentations do not have apache license headers, while the other files have.

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

No

### How was this patch tested?

`make html`

Closes #33979 from yoda-mon/add-license-header-makefiles.

Authored-by: Leona Yoda <yodal@oss.nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit a440025f08)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-14 09:16:39 +09:00
Minchu Yang 79e148ee93 [SPARK-36705][SHUFFLE] Disable push based shuffle when IO encryption is enabled or serializer is not relocatable
### What changes were proposed in this pull request?

Disable push-based shuffle when IO encryption is enabled or serializer does not support relocation of serialized objects.

### Why are the changes needed?

Push based shuffle is not compatible with IO encryption or non-relocatable serialization.

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

No.

### How was this patch tested?

Added some tests to check whether push-based shuffle can be disabled successfully when IO encryption is enabled or a serializer that does not support relocation of serialized object is used.

Closes #33976 from rmcyang/SPARK-36705.

Authored-by: Minchu Yang <minyang@minyang-mn3.linkedin.biz>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 999473b1a5)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-09-13 16:15:42 -05:00
Lukas Rytz 2e7583799e [SPARK-36712][BUILD] Make scala-parallel-collections in 2.13 POM a direct dependency (not in maven profile)
As [reported on `devspark.apache.org`](https://lists.apache.org/thread.html/r84cff66217de438f1389899e6d6891b573780159cd45463acf3657aa%40%3Cdev.spark.apache.org%3E), the published POMs when building with Scala 2.13 have the `scala-parallel-collections` dependency only in the `scala-2.13` profile of the pom.

### What changes were proposed in this pull request?

This PR suggests to work around this by un-commenting the `scala-parallel-collections` dependency when switching to 2.13 using the the `change-scala-version.sh` script.

I included an upgrade to scala-parallel-collections version 1.0.3, the changes compared to 0.2.0 are minor.
  - removed OSGi metadata
  - renamed some internal inner classes
  - added `Automatic-Module-Name`

### Why are the changes needed?

According to the posts, this solves issues for developers that write unit tests for their applications.

Stephen Coy suggested to use the https://www.mojohaus.org/flatten-maven-plugin. While this sounds like a more principled solution, it is possibly too risky to do at this specific point in time?

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

No

### How was this patch tested?

Locally

Closes #33948 from lrytz/parCollDep.

Authored-by: Lukas Rytz <lukas.rytz@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
(cherry picked from commit 1a62e6a2c1)
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-09-13 11:06:58 -05:00
Yuto Akutsu b043ee4de7 [SPARK-36738][SQL][DOC] Fixed the wrong documentation on Cot API
### What changes were proposed in this pull request?

Fixed wrong documentation on Cot API

### Why are the changes needed?

[Doc](https://spark.apache.org/docs/latest/api/sql/index.html#cot) says `1/java.lang.Math.cot` but it should be `1/java.lang.Math.tan`.

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

No.

### How was this patch tested?

Manual check.

Closes #33978 from yutoacts/SPARK-36738.

Authored-by: Yuto Akutsu <yuto.akutsu@nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 3747cfdb40)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-13 21:51:43 +09:00
Kousuke Saruta dad566c1f2 [SPARK-36729][BUILD] Upgrade Netty from 4.1.63 to 4.1.68
### What changes were proposed in this pull request?

This PR upgrades Netty from `4.1.63` to `4.1.68`.

All the changes from `4.1.64` to `4.1.68` are as follows.

* 4.1.64 and 4.1.65
  * https://netty.io/news/2021/05/19/4-1-65-Final.html
* 4.1.66
  * https://netty.io/news/2021/07/16/4-1-66-Final.html
* 4.1.67
  * https://netty.io/news/2021/08/16/4-1-67-Final.html
* 4.1.68
  * https://netty.io/news/2021/09/09/4-1-68-Final.html

### Why are the changes needed?

Recently Netty `4.1.68` was released, which includes official M1 Mac support.
* Add support for mac m1
  * https://github.com/netty/netty/pull/11666

`4.1.65` also includes a critical bug fix which Spark might be affected.
* JNI classloader deadlock with latest JDK version
  * https://github.com/netty/netty/issues/11209

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

No.

### How was this patch tested?

CIs.

Closes #33970 from sarutak/upgrade-netty-4.1.68.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit e1e19619b7)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-12 10:07:40 -07:00
yangjie01 90c4cf3cf3 [SPARK-36636][CORE][TEST] LocalSparkCluster change to use tmp workdir in test to avoid directory name collision
### What changes were proposed in this pull request?
As described in SPARK-36636,if the test cases with config `local-cluster[n, c, m]`  are run continuously within 1 second, the workdir name collision will occur because appid use format as `app-yyyyMMddHHmmss-0000` and workdir name associated with it  in test now,  the related logs are as follows:

```
java.io.IOException: Failed to create directory /spark-mine/work/app-20210908074432-0000/1
	at org.apache.spark.deploy.worker.Worker$$anonfun$receive$1.applyOrElse(Worker.scala:578)
	at org.apache.spark.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115)
	at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:213)
	at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:100)
	at org.apache.spark.rpc.netty.MessageLoop.org$apache$spark$rpc$netty$MessageLoop$$receiveLoop(MessageLoop.scala:75)
	at org.apache.spark.rpc.netty.MessageLoop$$anon$1.run(MessageLoop.scala:41)
	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)
21/09/08 22:44:32.266 dispatcher-event-loop-0 INFO Worker: Asked to launch executor app-20210908074432-0000/0 for test
21/09/08 22:44:32.266 dispatcher-event-loop-0 ERROR Worker: Failed to launch executor app-20210908074432-0000/0 for test.
java.io.IOException: Failed to create directory /spark-mine/work/app-20210908074432-0000/0
	at org.apache.spark.deploy.worker.Worker$$anonfun$receive$1.applyOrElse(Worker.scala:578)
	at org.apache.spark.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115)
	at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:213)
	at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:100)
	at org.apache.spark.rpc.netty.MessageLoop.org$apache$spark$rpc$netty$MessageLoop$$receiveLoop(MessageLoop.scala:75)
	at org.apache.spark.rpc.netty.MessageLoop$$anon$1.run(MessageLoop.scala:41)
	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)
```

Since the default value of s`park.deploy.maxExecutorRetries` is 10, the test failed will occur when 5 consecutive cases with local-cluster[3, 1, 1024] are completed within 1 second:

1. case 1: use worker directories: `/app-202109102324-0000/0`, `/app-202109102324-0000/1`, `/app-202109102324-0000/2`
2. case 2: retry 3 times then use worker directories: `/app-202109102324-0000/3`, `/app-202109102324-0000/4`, `/app-202109102324-0000/5`
3. case 3: retry 6 times then use worker directories: `/app-202109102324-0000/6`, `/app-202109102324-0000/7`, `/app-202109102324-0000/8`
4. case 4: retry 9 times then use worker directories: `/app-202109102324-0000/9`, `/app-202109102324-0000/10`, `/app-202109102324-0000/11`
5. case 5: retry more than **10** times then **failed**

To avoid this issue, this pr change to use tmp workdir in test with  config `local-cluster[n, c, m]`.

### Why are the changes needed?
Avoid UT failures caused by continuous workdir name collision.

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

### How was this patch tested?

- Pass GA or Jenkins Tests.
- Manual test: `build/mvn clean install -Pscala-2.13 -pl core -am` or `build/mvn clean install -pl core -am`, with Scala 2.13 is easier to reproduce this problem

**Before**

The test failed error logs as follows and randomness in test failure:
```
- SPARK-33084: Add jar support Ivy URI -- test exclude param when transitive=true *** FAILED ***
  org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.SparkContextSuite.$anonfun$new$134(SparkContextSuite.scala:1101)
org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
org.scalatest.Transformer.apply(Transformer.scala:22)
org.scalatest.Transformer.apply(Transformer.scala:20)
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2647)
  at scala.Option.foreach(Option.scala:437)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2644)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2734)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:95)
  at org.apache.spark.SparkContextSuite.$anonfun$new$138(SparkContextSuite.scala:1109)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
  at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
  at org.scalatest.Transformer.apply(Transformer.scala:22)
  ...
- SPARK-33084: Add jar support Ivy URI -- test different version *** FAILED ***
  org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.SparkContextSuite.$anonfun$new$134(SparkContextSuite.scala:1101)
org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
org.scalatest.Transformer.apply(Transformer.scala:22)
org.scalatest.Transformer.apply(Transformer.scala:20)
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2647)
  at scala.Option.foreach(Option.scala:437)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2644)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2734)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:95)
  at org.apache.spark.SparkContextSuite.$anonfun$new$142(SparkContextSuite.scala:1118)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
  at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
  at org.scalatest.Transformer.apply(Transformer.scala:22)
  ...
- SPARK-33084: Add jar support Ivy URI -- test invalid param *** FAILED ***
  org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.SparkContextSuite.$anonfun$new$134(SparkContextSuite.scala:1101)
org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
org.scalatest.Transformer.apply(Transformer.scala:22)
org.scalatest.Transformer.apply(Transformer.scala:20)
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2647)
  at scala.Option.foreach(Option.scala:437)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2644)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2734)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:95)
  at org.apache.spark.SparkContextSuite.$anonfun$new$146(SparkContextSuite.scala:1129)
  at org.apache.spark.SparkFunSuite.withLogAppender(SparkFunSuite.scala:235)
  at org.apache.spark.SparkContextSuite.$anonfun$new$145(SparkContextSuite.scala:1127)
  at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  ...
- SPARK-33084: Add jar support Ivy URI -- test multiple transitive params *** FAILED ***
  org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.SparkContextSuite.$anonfun$new$134(SparkContextSuite.scala:1101)
org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
org.scalatest.Transformer.apply(Transformer.scala:22)
org.scalatest.Transformer.apply(Transformer.scala:20)
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2647)
  at scala.Option.foreach(Option.scala:437)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2644)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2734)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:95)
  at org.apache.spark.SparkContextSuite.$anonfun$new$149(SparkContextSuite.scala:1140)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
  at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
  at org.scalatest.Transformer.apply(Transformer.scala:22)
  ...
- SPARK-33084: Add jar support Ivy URI -- test param key case sensitive *** FAILED ***
  org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.SparkContextSuite.$anonfun$new$134(SparkContextSuite.scala:1101)
org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
org.scalatest.Transformer.apply(Transformer.scala:22)
org.scalatest.Transformer.apply(Transformer.scala:20)
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2647)
  at scala.Option.foreach(Option.scala:437)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2644)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2734)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:95)
  at org.apache.spark.SparkContextSuite.$anonfun$new$154(SparkContextSuite.scala:1155)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
  at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
  at org.scalatest.Transformer.apply(Transformer.scala:22)
  ...
- SPARK-33084: Add jar support Ivy URI -- test transitive value case insensitive *** FAILED ***
  org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.SparkContextSuite.$anonfun$new$134(SparkContextSuite.scala:1101)
org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
org.scalatest.Transformer.apply(Transformer.scala:22)
org.scalatest.Transformer.apply(Transformer.scala:20)
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2647)
  at scala.Option.foreach(Option.scala:437)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2644)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2734)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:95)
  at org.apache.spark.SparkContextSuite.$anonfun$new$159(SparkContextSuite.scala:1166)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
  at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
  at org.scalatest.Transformer.apply(Transformer.scala:22)

```

**After**

```
Run completed in 26 minutes, 38 seconds.
Total number of tests run: 2863
Suites: completed 276, aborted 0
Tests: succeeded 2863, failed 0, canceled 4, ignored 8, pending 0
All tests passed.
```

Closes #33963 from LuciferYang/SPARK-36636.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
(cherry picked from commit 0e1157df06)
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-09-12 09:57:18 -05:00
dgd_contributor 6b62459c09 [SPARK-35822][UI] Spark UI-Executor tab is empty in IE11
### What changes were proposed in this pull request?
Refactor some functions in utils.js to fix the empty UI-Executor tab in yarn mode in IE11.

### Why are the changes needed?
Spark UI-Executor tab is empty in IE11: So this PR to fix this.
![Executortab_IE](https://user-images.githubusercontent.com/84778052/132786964-b17b6d12-457f-4ba3-894f-3f2e1c285b1e.PNG)

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

### How was this patch tested?
Existed UT Testcase

Closes #33937 from dgd-contributor/SPARK-35822-v2.

Authored-by: dgd_contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit ebca01f03e)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-11 15:58:42 -07:00
Kousuke Saruta b8a23e9ccc [SPARK-36725][SQL][TESTS] Ensure HiveThriftServer2Suites to stop Thrift JDBC server on exit
### What changes were proposed in this pull request?

This PR aims to ensure that HiveThriftServer2Suites (e.g. `thriftserver.UISeleniumSuite`) stop Thrift JDBC server on exit using shutdown hook.

### Why are the changes needed?

Normally, HiveThriftServer2Suites stops Thrift JDBC server via `afterAll` method.
But, if they are killed by signal (e.g. Ctrl-C), Thrift JDBC server will be remain.
```
$ jps
2792969 SparkSubmit
```
### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Killed `thriftserver.UISeleniumSuite` by Ctrl-C and confirmed no Thrift JDBC server is remain by jps.

Closes #33967 from sarutak/stop-thrift-on-exit.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit c36d70836d)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-11 15:54:48 -07:00
dgd-contributor d71e2fd1d1 [SPARK-36685][ML][MLLIB] Fix wrong assert messages
### What changes were proposed in this pull request?
Fix wrong assert statement, a mistake when coding

### Why are the changes needed?
 wrong assert statement

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

### How was this patch tested?
 Existing tests

Closes #33953 from dgd-contributor/SPARK-36685.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 9af0132516)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-11 14:39:55 -07:00
Sean Owen 043dd531a0 [SPARK-36704][CORE] Expand exception handling to more Java 9 cases where reflection is limited at runtime, when reflecting to manage DirectByteBuffer settings
### What changes were proposed in this pull request?

Improve exception handling in the Platform initialization, where it attempts to assess whether reflection is possible to modify DirectByteBuffer. This can apparently fail in more cases on Java 9+ than are currently handled, whereas Spark can continue without reflection if needed.

More detailed comments on the change inline.

### Why are the changes needed?

This exception seems to be possible and fails startup:

```
Caused by: java.lang.reflect.InaccessibleObjectException: Unable to make private java.nio.DirectByteBuffer(long,int) accessible: module java.base does not "opens java.nio" to unnamed module 71e9ddb4
        at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:357)
        at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:297)
        at java.base/java.lang.reflect.Constructor.checkCanSetAccessible(Constructor.java:188)
        at java.base/java.lang.reflect.Constructor.setAccessible(Constructor.java:181)
        at org.apache.spark.unsafe.Platform.<clinit>(Platform.java:56)
```

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

Should strictly allow Spark to continue in more cases.

### How was this patch tested?

Existing tests.

Closes #33947 from srowen/SPARK-36704.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
(cherry picked from commit e5283f5ed5)
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-09-11 13:38:20 -05:00
yi.wu dcb57ab42b Revert "[SPARK-35011][CORE] Avoid Block Manager registrations when StopExecutor msg is in-flight"
This reverts commit b9e53f8937.

### What changes were proposed in this pull request?

Revert https://github.com/apache/spark/pull/32114

### Why are the changes needed?

It breaks the expected `BlockManager` re-registration (e.g., heartbeat loss of an active executor) due to deferred removal of `BlockManager`, see the check:
9cefde8db3/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala (L551)

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

No

### How was this patch tested?

Pass existing tests.

Closes #33959 from Ngone51/revert-35011-3.2.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-10 09:29:55 -07:00
Liang-Chi Hsieh b52fbeee2d [SPARK-36669][SQL] Add Lz4 wrappers for Hadoop Lz4 codec
### What changes were proposed in this pull request?

This patch proposes to add a few LZ4 wrapper classes for Parquet Lz4 compression output that uses Hadoop Lz4 codec.

### Why are the changes needed?

Currently we use Hadop 3.3.1's shaded client libraries. Lz4 is a provided dependency in Hadoop Common 3.3.1 for Lz4Codec. But it isn't excluded from relocation in these libraries. So to use lz4 as Parquet codec, we will hit the exception even we include lz4 as dependency.

```
[info]   Cause: java.lang.NoClassDefFoundError: org/apache/hadoop/shaded/net/jpountz/lz4/LZ4Factory
[info]   at org.apache.hadoop.io.compress.lz4.Lz4Compressor.<init>(Lz4Compressor.java:66)
[info]   at org.apache.hadoop.io.compress.Lz4Codec.createCompressor(Lz4Codec.java:119)
[info]   at org.apache.hadoop.io.compress.CodecPool.getCompressor(CodecPool.java:152)
[info]   at org.apache.hadoop.io.compress.CodecPool.getCompressor(CodecPool.java:168)
```

Before the issue is fixed at Hadoop new release, we can add a few wrapper classes for Lz4 codec.

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

No

### How was this patch tested?

Modified test.

Closes #33940 from viirya/lz4-wrappers.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 6bcf330191)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-09 09:31:11 -07:00
Liang-Chi Hsieh d22182e474 [SPARK-34479][SQL][DOC][FOLLOWUP] Add zstandard to avro supported codecs
### What changes were proposed in this pull request?

Adding `zstandard` to avro supported codecs.

### Why are the changes needed?

To improve the document.

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

No

### How was this patch tested?

Doc only.

Closes #33943 from viirya/minor-doc.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 647ffe655f)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-08 23:21:38 -07:00
Liang-Chi Hsieh 2d75b99b97 [SPARK-36670][SQL][TEST][FOLLOWUP] Add AvroCodecSuite
### What changes were proposed in this pull request?

This patch proposes to add also `AvroCodecSuite` as a follow up for SPARK-36670.

### Why are the changes needed?

Improve test coverage.

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

No

### How was this patch tested?

Added test.

Closes #33939 from viirya/SPARK-36670-avro.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit c95d3fe6c9)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-08 22:17:49 -07:00
Andrew Liu 6cb23c163c [SPARK-36686][SQL] Fix SimplifyConditionalsInPredicate to be null-safe
### What changes were proposed in this pull request?

fix SimplifyConditionalsInPredicate to be null-safe

Reproducible:

```
import org.apache.spark.sql.types.{StructField, BooleanType, StructType}
import org.apache.spark.sql.Row

val schema = List(
  StructField("b", BooleanType, true)
)
val data = Seq(
  Row(true),
  Row(false),
  Row(null)
)
val df = spark.createDataFrame(
  spark.sparkContext.parallelize(data),
  StructType(schema)
)

// cartesian product of true / false / null
val df2 = df.select(col("b") as "cond").crossJoin(df.select(col("b") as "falseVal"))
df2.createOrReplaceTempView("df2")

spark.sql("SELECT * FROM df2 WHERE IF(cond, FALSE, falseVal)").show()
// actual:
// +-----+--------+
// | cond|falseVal|
// +-----+--------+
// |false|    true|
// +-----+--------+
spark.sql("SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.SimplifyConditionalsInPredicate")
spark.sql("SELECT * FROM df2 WHERE IF(cond, FALSE, falseVal)").show()
// expected:
// +-----+--------+
// | cond|falseVal|
// +-----+--------+
// |false|    true|
// | null|    true|
// +-----+--------+
```

### Why are the changes needed?

is a regression that leads to incorrect results

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

no

### How was this patch tested?

existing tests

Closes #33928 from hypercubestart/fix-SimplifyConditionalsInPredicate.

Authored-by: Andrew Liu <andrewlliu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 9b633f2075)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-09 11:32:59 +08:00
Xinrong Meng 88bba0c94b [SPARK-36697][PYTHON] Fix dropping all columns of a DataFrame
### What changes were proposed in this pull request?
Fix dropping all columns of a DataFrame

### Why are the changes needed?
When dropping all columns of a pandas-on-Spark DataFrame, a ValueError is raised.
Whereas in pandas, an empty DataFrame reserving the index is returned.
We should follow pandas.

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

From
```py
>>> psdf = ps.DataFrame({"x": [1, 2], "y": [3, 4], "z": [5, 6]})
>>> psdf
   x  y  z
0  1  3  5
1  2  4  6

>>> psdf.drop(['x', 'y', 'z'])
Traceback (most recent call last):
...
ValueError: not enough values to unpack (expected 2, got 0)

```
To
```py
>>> psdf = ps.DataFrame({"x": [1, 2], "y": [3, 4], "z": [5, 6]})
>>> psdf
   x  y  z
0  1  3  5
1  2  4  6

>>> psdf.drop(['x', 'y', 'z'])
Empty DataFrame
Columns: []
Index: [0, 1]
```

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

Closes #33938 from xinrong-databricks/frame_drop_col.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 33bb7b39e9)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-09 09:59:50 +09:00
Huaxin Gao 7e8860751c [SPARK-34952][SQL][FOLLOWUP] Change column type to be NamedReference
### What changes were proposed in this pull request?
Currently, we have `FieldReference` for aggregate column type, should be `NamedReference` instead

### Why are the changes needed?
`FieldReference` is a private class, should use `NamedReference` instead

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

### How was this patch tested?
existing tests

Closes #33927 from huaxingao/agg_followup.

Authored-by: Huaxin Gao <huaxin_gao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 23794fb303)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-08 14:06:26 +08:00
yangjie01 c4332c7bf0 [SPARK-36684][SQL][TESTS] Add Jackson test dependencies to sql/core module at hadoop-2.7 profile
### What changes were proposed in this pull request?
SPARK-26346 upgrade Parquet related modules from 1.10.1 to 1.11.1 and `parquet-jackson 1.11.1` use `com.fasterxml.jackson` instead of `org.codehaus.jackson`.

So, there are warning logs related to

```
17:12:17.605 WARN org.apache.hadoop.fs.FileSystem: Cannot load filesystem
java.util.ServiceConfigurationError: org.apache.hadoop.fs.FileSystem: Provider org.apache.hadoop.hdfs.web.WebHdfsFileSystem could not be instantiated
...
Caused by: java.lang.ClassNotFoundException: org.codehaus.jackson.map.ObjectMapper
	at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
...
```

when test `sql/core` modules with `hadoop-2.7` profile.

This pr adds test dependencies related to `org.codehaus.jackson` in `sql/core` module when `hadoop-2.7` profile is activated.

### Why are the changes needed?
Clean up test warning logs that shouldn't exist.

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

### How was this patch tested?

- Pass GA or Jenkins Tests.
- Manual test `mvn clean test -pl sql/core -am -DwildcardSuites=none -Phadoop-2.7`

**Before**

No test failed, but warning logs as follows:

```
[INFO] Running test.org.apache.spark.sql.JavaBeanDeserializationSuite
22:42:45.211 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
22:42:46.827 WARN org.apache.hadoop.fs.FileSystem: Cannot load filesystem
java.util.ServiceConfigurationError: org.apache.hadoop.fs.FileSystem: Provider org.apache.hadoop.hdfs.web.WebHdfsFileSystem could not be instantiated
	at java.util.ServiceLoader.fail(ServiceLoader.java:232)
	at java.util.ServiceLoader.access$100(ServiceLoader.java:185)
	at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384)
	at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
	at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
	at org.apache.hadoop.fs.FileSystem.loadFileSystems(FileSystem.java:2631)
	at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2650)
	at org.apache.hadoop.fs.FsUrlStreamHandlerFactory.<init>(FsUrlStreamHandlerFactory.java:62)
	at org.apache.spark.sql.internal.SharedState$.liftedTree1$1(SharedState.scala:181)
	at org.apache.spark.sql.internal.SharedState$.org$apache$spark$sql$internal$SharedState$$setFsUrlStreamHandlerFactory(SharedState.scala:180)
	at org.apache.spark.sql.internal.SharedState.<init>(SharedState.scala:54)
	at org.apache.spark.sql.SparkSession.$anonfun$sharedState$1(SparkSession.scala:135)
	at scala.Option.getOrElse(Option.scala:189)
	at org.apache.spark.sql.SparkSession.sharedState$lzycompute(SparkSession.scala:135)
	at org.apache.spark.sql.SparkSession.sharedState(SparkSession.scala:134)
	at org.apache.spark.sql.internal.BaseSessionStateBuilder.build(BaseSessionStateBuilder.scala:335)
	at org.apache.spark.sql.test.TestSparkSession.sessionState$lzycompute(TestSQLContext.scala:42)
	at org.apache.spark.sql.test.TestSparkSession.sessionState(TestSQLContext.scala:41)
	at org.apache.spark.sql.SparkSession.$anonfun$new$3(SparkSession.scala:109)
	at scala.Option.map(Option.scala:230)
	at org.apache.spark.sql.SparkSession.$anonfun$new$1(SparkSession.scala:109)
	at org.apache.spark.sql.internal.SQLConf$.get(SQLConf.scala:194)
	at org.apache.spark.sql.types.DataType.sameType(DataType.scala:97)
	at org.apache.spark.sql.catalyst.analysis.TypeCoercion$.$anonfun$haveSameType$1(TypeCoercion.scala:291)
	at org.apache.spark.sql.catalyst.analysis.TypeCoercion$.$anonfun$haveSameType$1$adapted(TypeCoercion.scala:291)
	at scala.collection.LinearSeqOptimized.forall(LinearSeqOptimized.scala:85)
	at scala.collection.LinearSeqOptimized.forall$(LinearSeqOptimized.scala:82)
	at scala.collection.immutable.List.forall(List.scala:89)
	at org.apache.spark.sql.catalyst.analysis.TypeCoercion$.haveSameType(TypeCoercion.scala:291)
	at org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.dataTypeCheck(Expression.scala:1074)
	at org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.dataTypeCheck$(Expression.scala:1069)
	at org.apache.spark.sql.catalyst.expressions.If.dataTypeCheck(conditionalExpressions.scala:37)
	at org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.org$apache$spark$sql$catalyst$expressions$ComplexTypeMergingExpression$$internalDataType(Expression.scala:1080)
	at org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.org$apache$spark$sql$catalyst$expressions$ComplexTypeMergingExpression$$internalDataType$(Expression.scala:1079)
	at org.apache.spark.sql.catalyst.expressions.If.org$apache$spark$sql$catalyst$expressions$ComplexTypeMergingExpression$$internalDataType$lzycompute(conditionalExpressions.scala:37)
	at org.apache.spark.sql.catalyst.expressions.If.org$apache$spark$sql$catalyst$expressions$ComplexTypeMergingExpression$$internalDataType(conditionalExpressions.scala:37)
	at org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.dataType(Expression.scala:1084)
	at org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.dataType$(Expression.scala:1084)
	at org.apache.spark.sql.catalyst.expressions.If.dataType(conditionalExpressions.scala:37)
	at org.apache.spark.sql.catalyst.expressions.objects.MapObjects.$anonfun$dataType$4(objects.scala:815)
	at scala.Option.getOrElse(Option.scala:189)
	at org.apache.spark.sql.catalyst.expressions.objects.MapObjects.dataType(objects.scala:815)
	at org.apache.spark.sql.catalyst.expressions.CreateNamedStruct.$anonfun$dataType$9(complexTypeCreator.scala:416)
	at scala.collection.immutable.List.map(List.scala:290)
	at org.apache.spark.sql.catalyst.expressions.CreateNamedStruct.dataType$lzycompute(complexTypeCreator.scala:410)
	at org.apache.spark.sql.catalyst.expressions.CreateNamedStruct.dataType(complexTypeCreator.scala:409)
	at org.apache.spark.sql.catalyst.expressions.CreateNamedStruct.dataType(complexTypeCreator.scala:398)
	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.isSerializedAsStruct(ExpressionEncoder.scala:309)
	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.isSerializedAsStructForTopLevel(ExpressionEncoder.scala:319)
	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.<init>(ExpressionEncoder.scala:248)
	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$.javaBean(ExpressionEncoder.scala:75)
	at org.apache.spark.sql.Encoders$.bean(Encoders.scala:154)
	at org.apache.spark.sql.Encoders.bean(Encoders.scala)
	at test.org.apache.spark.sql.JavaBeanDeserializationSuite.testBeanWithArrayFieldDeserialization(JavaBeanDeserializationSuite.java:75)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
	at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
	at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
	at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
	at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
	at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
	at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
	at org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:364)
	at org.apache.maven.surefire.junit4.JUnit4Provider.executeWithRerun(JUnit4Provider.java:272)
	at org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:237)
	at org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:158)
	at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:428)
	at org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:162)
	at org.apache.maven.surefire.booter.ForkedBooter.run(ForkedBooter.java:562)
	at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:548)
Caused by: java.lang.NoClassDefFoundError: org/codehaus/jackson/map/ObjectMapper
	at org.apache.hadoop.hdfs.web.WebHdfsFileSystem.<clinit>(WebHdfsFileSystem.java:129)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
	at java.lang.Class.newInstance(Class.java:442)
	at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:380)
	... 81 more
Caused by: java.lang.ClassNotFoundException: org.codehaus.jackson.map.ObjectMapper
	at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:419)
	at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:352)
	... 88 more
```

**After**

There are no more warning logs like above

Closes #33926 from LuciferYang/SPARK-36684.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
(cherry picked from commit acd9c92fa8)
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-09-07 21:40:53 -07:00
itholic 3d50760a3e [SPARK-36531][SPARK-36515][PYTHON] Improve test coverage for data_type_ops/* and groupby
### What changes were proposed in this pull request?

This PR proposes improving test coverage for pandas-on-Spark data types & GroupBy code base, which is written in `data_type_ops/*.py` and `groupby.py` separately.

This PR did the following to improve coverage:
- Add unittest for untested code
- Fix unittest which is not tested properly
- Remove unused code

**NOTE**: This PR is not only include the test-only update, for example it includes the fixing `astype` for binary ops.

pandas-on-Spark Series we have:
```python
>>> psser
0    [49]
1    [50]
2    [51]
dtype: object
```

before:
```python
>>> psser.astype(bool)
Traceback (most recent call last):
...
pyspark.sql.utils.AnalysisException: cannot resolve 'CAST(`0` AS BOOLEAN)' due to data type mismatch: cannot cast binary to boolean;
...
```

after:
```python
>>> psser.astype(bool)
0    True
1    True
2    True
dtype: bool
```

### Why are the changes needed?

To make the project healthier by improving coverage.

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

No.

### How was this patch tested?

Unittest.

Closes #33850 from itholic/SPARK-36531.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 71dbd03fbe)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-08 10:23:06 +09:00
Venkata Sai Akhil Gudesa 4a92b0e278 [SPARK-36677][SQL] NestedColumnAliasing should not push down aggregate functions into projections
### What changes were proposed in this pull request?

This PR filters out `ExtractValues`s that contains any aggregation function in the `NestedColumnAliasing` rule to prevent cases where aggregations are pushed down into projections.

### Why are the changes needed?

To handle a corner/missed case in `NestedColumnAliasing` that can cause users to encounter a runtime exception.

Consider the following schema:
```
root
 |-- a: struct (nullable = true)
 |    |-- c: struct (nullable = true)
 |    |    |-- e: string (nullable = true)
 |    |-- d: integer (nullable = true)
 |-- b: string (nullable = true)
```
and the query:
`SELECT MAX(a).c.e FROM (SELECT a, b FROM test_aggregates) GROUP BY b`

Executing the query before this PR will result in the error:
```
java.lang.UnsupportedOperationException: Cannot generate code for expression: max(input[0, struct<c:struct<e:string>,d:int>, true])
  at org.apache.spark.sql.errors.QueryExecutionErrors$.cannotGenerateCodeForExpressionError(QueryExecutionErrors.scala:83)
  at org.apache.spark.sql.catalyst.expressions.Unevaluable.doGenCode(Expression.scala:312)
  at org.apache.spark.sql.catalyst.expressions.Unevaluable.doGenCode$(Expression.scala:311)
  at org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression.doGenCode(interfaces.scala:99)
...
```
The optimised plan before this PR is:

```
'Aggregate [b#1], [_extract_e#5 AS max(a).c.e#3]
+- 'Project [max(a#0).c.e AS _extract_e#5, b#1]
   +- Relation default.test_aggregates[a#0,b#1] parquet
```

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

No

### How was this patch tested?

A new unit test in `NestedColumnAliasingSuite`. The test consists of the repro mentioned earlier.
The produced optimized plan is checked for equivalency with a plan of the form:
```
 Aggregate [b#452], [max(a#451).c.e AS max('a)[c][e]#456]
+- LocalRelation <empty>, [a#451, b#452]
```

Closes #33921 from vicennial/spark-36677.

Authored-by: Venkata Sai Akhil Gudesa <venkata.gudesa@databricks.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 2ed6e7bc5d)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-07 18:16:02 -07:00
Liang-Chi Hsieh e39948fada [SPARK-36670][SQL][TEST] Add FileSourceCodecSuite
### What changes were proposed in this pull request?

This patch mainly proposes to add some e2e test cases in Spark for codec used by main datasources.

### Why are the changes needed?

We found there is no e2e test cases available for main datasources like Parquet, Orc. It makes developers harder to identify possible bugs early. We should add such tests in Spark.

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

No

### How was this patch tested?

Added tests.

Closes #33912 from viirya/SPARK-36670.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 5a0ae694d0)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-07 16:53:25 -07:00
Andy Grove 533f655690 [SPARK-36666][SQL] Fix regression in AQEShuffleReadExec
Fix regression in AQEShuffleReadExec when used in conjunction with Spark plugins with custom partitioning.

Signed-off-by: Andy Grove <andygrove73gmail.com>

### What changes were proposed in this pull request?

Return `UnknownPartitioning` rather than throw an exception in `AQEShuffleReadExec`.

### Why are the changes needed?

The [RAPIDS Accelerator for Apache Spark](https://github.com/NVIDIA/spark-rapids) replaces `AQEShuffleReadExec` with a custom operator that runs on the GPU. Due to changes in [SPARK-36315](dd80457ffb), Spark now throws an exception if the shuffle exchange does not have recognized partitioning, and this happens before the postStageOptimizer rules so there is no opportunity to replace this operator now.

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

No.

### How was this patch tested?

I am still in the process of testing this change. I will update the PR in the next few days with status.

Closes #33910 from andygrove/SPARK-36666.

Authored-by: Andy Grove <andygrove73@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
(cherry picked from commit f78d8394dc)
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-09-07 13:50:00 -07:00
Liang-Chi Hsieh acd3856215 [SPARK-36682][CORE][TEST] Add Hadoop sequence file test for different Hadoop codecs
### What changes were proposed in this pull request?

This patch proposes to add e2e tests for using Hadoop codecs to write sequence files.

### Why are the changes needed?

To improve test coverage.

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

No

### How was this patch tested?

Added tests.

Closes #33924 from viirya/hadoop-seq-test.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 6745d77818)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-07 13:20:09 -07:00
Kousuke Saruta a41dc4516e [SPARK-36153][SQL][DOCS][FOLLOWUP] Fix the description about the possible values of spark.sql.catalogImplementation property
### What changes were proposed in this pull request?

This PR fixes the description about the possible values of `spark.sql.catalogImplementation` property.
It was added in SPARK-36153 (#33362) but the possible values are `hive` or `in-memory` rather than `true` or `false`.

### Why are the changes needed?

To fix wrong description.

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

No.

### How was this patch tested?

I just confirmed `in-memory` and `hive` are the valid values with SparkShell.

Closes #33923 from sarutak/fix-doc-about-catalogImplementation.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit a5fe5d368c)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-07 11:39:53 +09:00
Jungtaek Lim e16c886b87 [SPARK-36667][SS][TEST] Close resources properly in StateStoreSuite/RocksDBStateStoreSuite
### What changes were proposed in this pull request?

This PR proposes to ensure StateStoreProvider instances are properly closed for each test in StateStoreSuite/RocksDBStateStoreSuite.

### Why are the changes needed?

While this doesn't break the test, this is a bad practice and may possibly make nasty problems in the future.

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

No.

### How was this patch tested?

Existing UTs

Closes #33916 from HeartSaVioR/SPARK-36667.

Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 093c2080fe)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-06 17:40:13 -07:00
Kent Yao aa96a374b2 [SPARK-36659][SQL] Promote spark.sql.execution.topKSortFallbackThreshold to a user-facing config
### What changes were proposed in this pull request?

Promote spark.sql.execution.topKSortFallbackThreshold to a user-facing config

### Why are the changes needed?

spark.sql.execution.topKSortFallbackThreshold now is an internal config hidden from users Integer.MAX_VALUE - 15 as its default. In many real-world cases, if the K is very big,  there would be performance issues.

It's better to leave this choice to users

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

 spark.sql.execution.topKSortFallbackThreshold is now user-facing

### How was this patch tested?

passing GA

Closes #33904 from yaooqinn/SPARK-36659.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
(cherry picked from commit 7f1ad7be18)
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-09-03 10:27:10 -07:00
Kousuke Saruta a3901ed384 [SPARK-36639][SQL] Fix an issue that sequence builtin function causes ArrayIndexOutOfBoundsException if the arguments are under the condition of start == stop && step < 0
### What changes were proposed in this pull request?

This PR fixes an issue that `sequence` builtin function causes `ArrayIndexOutOfBoundsException` if the arguments are under the condition of `start == stop && step < 0`.
This is an example.
```
SELECT sequence(timestamp'2021-08-31', timestamp'2021-08-31', -INTERVAL 1 month);
21/09/02 04:14:42 ERROR SparkSQLDriver: Failed in [SELECT sequence(timestamp'2021-08-31', timestamp'2021-08-31', -INTERVAL 1 month)]
java.lang.ArrayIndexOutOfBoundsException: 1
```
Actually, this example succeeded before SPARK-31980 (#28819) was merged.

### Why are the changes needed?

Bug fix.

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

No.

### How was this patch tested?

New tests.

Closes #33895 from sarutak/fix-sequence-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
(cherry picked from commit cf3bc65e69)
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-09-03 23:25:33 +09:00