Commit graph

31072 commits

Author SHA1 Message Date
Hyukjin Kwon 9eec11b956 [SPARK-36379][SQL] Null at root level of a JSON array should not fail w/ permissive mode
This PR proposes to fail properly so JSON parser can proceed and parse the input with the permissive mode.
Previously, we passed `null`s as are, the root `InternalRow`s became `null`s, and it causes the query fails even with permissive mode on.
Now, we fail explicitly if `null` is passed when the input array contains `null`.

Note that this is consistent with non-array JSON input:

**Permissive mode:**

```scala
spark.read.json(Seq("""{"a": "str"}""", """null""").toDS).collect()
```
```
res0: Array[org.apache.spark.sql.Row] = Array([str], [null])
```

**Failfast mode**:

```scala
spark.read.option("mode", "failfast").json(Seq("""{"a": "str"}""", """null""").toDS).collect()
```
```
org.apache.spark.SparkException: Malformed records are detected in record parsing. Parse Mode: FAILFAST. To process malformed records as null result, try setting the option 'mode' as 'PERMISSIVE'.
	at org.apache.spark.sql.catalyst.util.FailureSafeParser.parse(FailureSafeParser.scala:70)
	at org.apache.spark.sql.DataFrameReader.$anonfun$json$7(DataFrameReader.scala:540)
	at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484)
```

To make the permissive mode to proceed and parse without throwing an exception.

**Permissive mode:**

```scala
spark.read.json(Seq("""[{"a": "str"}, null]""").toDS).collect()
```

Before:

```
java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
```

After:

```
res0: Array[org.apache.spark.sql.Row] = Array([null])
```

NOTE that this behaviour is consistent when JSON object is malformed:

```scala
spark.read.schema("a int").json(Seq("""[{"a": 123}, {123123}, {"a": 123}]""").toDS).collect()
```

```
res0: Array[org.apache.spark.sql.Row] = Array([null])
```

Since we're parsing _one_ JSON array, related records all fail together.

**Failfast mode:**

```scala
spark.read.option("mode", "failfast").json(Seq("""[{"a": "str"}, null]""").toDS).collect()
```

Before:

```
java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
```

After:

```
org.apache.spark.SparkException: Malformed records are detected in record parsing. Parse Mode: FAILFAST. To process malformed records as null result, try setting the option 'mode' as 'PERMISSIVE'.
	at org.apache.spark.sql.catalyst.util.FailureSafeParser.parse(FailureSafeParser.scala:70)
	at org.apache.spark.sql.DataFrameReader.$anonfun$json$7(DataFrameReader.scala:540)
	at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484)
```

Manually tested, and unit test was added.

Closes #33608 from HyukjinKwon/SPARK-36379.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 0bbcbc6508)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-02 10:02:09 -07:00
Angerszhuuuu ea559adc2e [SPARK-36086][SQL] CollapseProject project replace alias should use origin column name
### What changes were proposed in this pull request?
For added UT, without this patch will failed as below
```
[info] - SHOW TABLES V2: SPARK-36086: CollapseProject project replace alias should use origin column name *** FAILED *** (4 seconds, 935 milliseconds)
[info]   java.lang.RuntimeException: After applying rule org.apache.spark.sql.catalyst.optimizer.CollapseProject in batch Operator Optimization before Inferring Filters, the structural integrity of the plan is broken.
[info]   at org.apache.spark.sql.errors.QueryExecutionErrors$.structuralIntegrityIsBrokenAfterApplyingRuleError(QueryExecutionErrors.scala:1217)
[info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:229)
[info]   at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
[info]   at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)
[info]   at scala.collection.immutable.List.foldLeft(List.scala:91)
[info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:208)
[info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:200)
[info]   at scala.collection.immutable.List.foreach(List.scala:431)
[info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:200)
[info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:179)
[info]   at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
```

CollapseProject project replace alias should use origin column name
### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT

Closes #33576 from AngersZhuuuu/SPARK-36086.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit f3173956cb)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-08-03 00:08:30 +08:00
Linhong Liu e26cb968bd [SPARK-36224][SQL] Use Void as the type name of NullType
### What changes were proposed in this pull request?
Change the `NullType.simpleString` to "void" to set "void" as the formal type name of `NullType`

### Why are the changes needed?
This PR is intended to address the type name discussion in PR #28833. Here are the reasons:
1. The type name of NullType is displayed everywhere, e.g. schema string, error message, document. Hence it's not possible to hide it from users, we have to choose a proper name
2. The "void" is widely used as the type name of "NULL", e.g. Hive, pgSQL
3. Changing to "void" can enable the round trip of `toDDL`/`fromDDL` for NullType. (i.e. make `from_json(col, schema.toDDL)`) work

### Does this PR introduce _any_ user-facing change?
Yes, the type name of "NULL" is changed from "null" to "void". for example:
```
scala> sql("select null as a, 1 as b").schema.catalogString
res5: String = struct<a:void,b:int>
```

### How was this patch tested?
existing test cases

Closes #33437 from linhongliu-db/SPARK-36224-void-type-name.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 2f700773c2)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-08-02 23:20:11 +08:00
yi.wu df43300227 [SPARK-36206][CORE] Support shuffle data corruption diagnosis via shuffle checksum
### What changes were proposed in this pull request?

This PR adds support to diagnose shuffle data corruption. Basically, the diagnosis mechanism works like this:
The shuffler reader would calculate the checksum (c1) for the corrupted shuffle block and send it to the server where the block is stored. At the server, it would read back the checksum (c2) that is stored in the checksum file and recalculate the checksum (c3) for the corresponding shuffle block. Then, if c2 != c3, we suspect the corruption is caused by the disk issue. Otherwise, if c1 != c3, we suspect the corruption is caused by the network issue. Otherwise, the checksum verifies pass. In any case of the error, the cause remains unknown.

After the shuffle reader receives the diagnosis response, it'd take the action bases on the type of cause. Only in case of the network issue, we'd give a retry. Otherwise, we'd throw the fetch failure directly. Also note that, if the corruption happens inside BufferReleasingInputStream, the reducer will throw the fetch failure immediately no matter what the cause is since the data has been partially consumed by downstream RDDs. If corruption happens again after retry, the reducer will throw the fetch failure directly this time without the diagnosis.

Please check out https://github.com/apache/spark/pull/32385 to see the completed proposal of the shuffle checksum project.

### Why are the changes needed?

Shuffle data corruption is a long-standing issue in Spark. For example, in SPARK-18105, people continually reports corruption issue. However, data corruption is difficult to reproduce in most cases and even harder to tell the root cause. We don't know if it's a Spark issue or not. With the diagnosis support for the shuffle corruption, Spark itself can at least distinguish the cause between disk and network, which is very important for users.

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

Yes, users may know the cause of the shuffle corruption after this change.

### How was this patch tested?

Added tests.

Closes #33451 from Ngone51/SPARK-36206.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit a98d919da4)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-02 09:59:30 -05:00
Terry Kim 87ae397897 [SPARK-36372][SQL] v2 ALTER TABLE ADD COLUMNS should check duplicates for the user specified columns
### What changes were proposed in this pull request?

Currently, v2 ALTER TABLE ADD COLUMNS does not check duplicates for the user specified columns. For example,
```
spark.sql(s"CREATE TABLE $t (id int) USING $v2Format")
spark.sql("ALTER TABLE $t ADD COLUMNS (data string, data string)")
```
doesn't fail the analysis, and it's up to the catalog implementation to handle it. For v1 command, the duplication is checked before invoking the catalog.

### Why are the changes needed?

To check the duplicate columns during analysis and be consistent with v1 command.

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

Yes, now the above will command will print out the fllowing:
```
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the user specified columns: `data`
```

### How was this patch tested?

Added new unit tests

Closes #33600 from imback82/alter_add_duplicate_columns.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 3b713e7f61)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-08-02 17:55:07 +08:00
Venkata krishnan Sowrirajan a9b32b390c [SPARK-32923][CORE][SHUFFLE] Handle indeterminate stage retries for push-based shuffle
[[SPARK-23243](https://issues.apache.org/jira/browse/SPARK-23243)] and [[SPARK-25341](https://issues.apache.org/jira/browse/SPARK-25341)] addressed cases of stage retries for indeterminate stage involving operations like repartition. This PR addresses the same issues in the context of push-based shuffle. Currently there is no way to distinguish the current execution of a stage for a shuffle ID. Therefore the changes explained below are necessary.

Core changes are summarized as follows:

1. Introduce a new variable `shuffleMergeId` in `ShuffleDependency` which is monotonically increasing value tracking the temporal ordering of execution of <stage-id, stage-attempt-id> for a shuffle ID.
2. Correspondingly make changes in the push-based shuffle protocol layer in `MergedShuffleFileManager`, `BlockStoreClient` passing the `shuffleMergeId` in order to keep track of the shuffle output in separate files on the shuffle service side.
3. `DAGScheduler` increments the `shuffleMergeId` tracked in `ShuffleDependency` in the cases of a indeterministic stage execution
4. Deterministic stage will have `shuffleMergeId` set to 0 as no special handling is needed in this case and indeterminate stage will have `shuffleMergeId` starting from 1.

New protocol changes are needed due to the reasons explained above.

No

Added new unit tests in `RemoteBlockPushResolverSuite, DAGSchedulerSuite, BlockIdSuite, ErrorHandlerSuite`

Closes #33034 from venkata91/SPARK-32923.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit c039d99812)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-01 23:30:02 -05:00
Venkata krishnan Sowrirajan cd3ab00382 [SPARK-32919][FOLLOW-UP] Filter out driver in the merger locations and fix the return type of RemoveShufflePushMergerLocations
### What changes were proposed in this pull request?

SPARK-32919 added support for fetching shuffle push merger locations with push-based shuffle. Filter out driver host in the shuffle push merger locations as driver won't participate in the shuffle merge also fix ClassCastException in the RemoveShufflePushMergerLocations.
### Why are the changes needed?

No

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

No

### How was this patch tested?

Added unit tests.

Closes #33425 from venkata91/SPARK-32919-follow-up.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 2a18f82940)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-01 13:12:22 -05:00
Hyukjin Kwon 310cd8eef1 [SPARK-36092][INFRA][BUILD][PYTHON] Migrate to GitHub Actions with Codecov from Jenkins
This PR proposes to migrate Coverage report from Jenkins to GitHub Actions by setting a dailly cron job.

For some background, currently PySpark code coverage is being reported in this specific Jenkins job: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/

Because of the security issue between [Codecov service](https://app.codecov.io/gh/) and Jenkins machines, we had to work around by manually hosting a coverage site via GitHub pages, see also https://spark-test.github.io/pyspark-coverage-site/ by spark-test account (which is shared to only subset of PMC members).

Since we now run the build via GitHub Actions, we can leverage [Codecov plugin](https://github.com/codecov/codecov-action), and remove the workaround we used.

Virtually no. Coverage site (UI) might change but the information it holds should be virtually the same.

I manually tested:
- Scheduled run: https://github.com/HyukjinKwon/spark/actions/runs/1082261484
- Coverage report: 73f0291a7d/python/pyspark
- Run against a PR: https://github.com/HyukjinKwon/spark/actions/runs/1082367175

Closes #33591 from HyukjinKwon/SPARK-36092.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit c0d1860f25)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-01 21:38:39 +09:00
Yikun Jiang 5548cd5a7f [SPARK-35976][PYTHON] Adjust astype method for ExtensionDtype in pandas API on Spark
### What changes were proposed in this pull request?
This patch set value to `<NA>` (pd.NA) in BooleanExtensionOps and StringExtensionOps.

### Why are the changes needed?
The pandas behavior:
```python
>>> pd.Series([True, False, None], dtype="boolean").astype(str).tolist()
['True', 'False', '<NA>']
>>> pd.Series(['s1', 's2', None], dtype="string").astype(str).tolist()
['1', '2', '<NA>']
```

pandas on spark
```python
>>> import pandas as pd
>>> from pyspark import pandas as ps

# Before
>>> ps.from_pandas(pd.Series([True, False, None], dtype="boolean")).astype(str).tolist()
['True', 'False', 'None']
>>> ps.from_pandas(pd.Series(['s1', 's2', None], dtype="string")).astype(str).tolist()
['True', 'False', 'None']

# After
>>> ps.from_pandas(pd.Series([True, False, None], dtype="boolean")).astype(str).tolist()
['True', 'False', '<NA>']
>>> ps.from_pandas(pd.Series(['s1', 's2', None], dtype="string")).astype(str).tolist()
['s1', 's2', '<NA>']
```

See more in [SPARK-35976](https://issues.apache.org/jira/browse/SPARK-35976)

### Does this PR introduce _any_ user-facing change?
Yes, return `<NA>` when None to follow the pandas behavior

### How was this patch tested?
Change the ut to cover this scenario.

Closes #33585 from Yikun/SPARK-35976.

Authored-by: Yikun Jiang <yikunkero@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit f04e991e6a)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-01 10:37:35 +09:00
Hyukjin Kwon 390aeb727b Revert "[SPARK-36345][INFRA] Update PySpark GitHubAction docker image to 20210730"
This reverts commit 586f89886f.
2021-08-01 10:35:32 +09:00
Takuya UESHIN 5d37a77d44 [SPARK-36365][PYTHON] Remove old workarounds related to null ordering
### What changes were proposed in this pull request?

Remove old workarounds related to null ordering.

### Why are the changes needed?

In pandas-on-Spark, there are still some remaining places to call `Column._jc.(asc|desc)_nulls_(first|last)` as a workaround from Koalas to support Spark 2.3.

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

No.

### How was this patch tested?

Modified a couple of tests and existing tests.

Closes #33597 from ueshin/issues/SPARK-36365/nulls_first_last.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 90d31dfcb7)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-01 10:33:43 +09:00
Hyukjin Kwon 187aa6ab7f [SPARK-36338][PYTHON][FOLLOW-UP] Keep the original default value as 'sequence' in default index in pandas on Spark
### What changes were proposed in this pull request?

This PR is a followup of https://github.com/apache/spark/pull/33570, which mistakenly changed the default value of the default index

### Why are the changes needed?

It was mistakenly changed. It was changed to check if the tests actually pass but I forgot to change it back.

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

No, it's not related yet. It fixes up the mistake of the default value mistakenly changed.
(Changed default value makes the test flaky because of the order affected by extra shuffle)

### How was this patch tested?

Manually tested.

Closes #33596 from HyukjinKwon/SPARK-36338-followup.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 74a6b9d23b)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-31 08:31:19 +09:00
Dongjoon Hyun 586f89886f [SPARK-36345][INFRA] Update PySpark GitHubAction docker image to 20210730
### What changes were proposed in this pull request?

This PR aims to upgrade PySpark GitHub Action job to use the latest docker image `20210730` having `sklearn` and `mlflow` additionally.
- 5ca94453d1

```
$ docker run -it --rm dongjoon/apache-spark-github-action-image:20210730 python3.9 -m pip list | grep mlflow
mlflow                    1.19.0

$ docker run -it --rm dongjoon/apache-spark-github-action-image:20210730 python3.9 -m pip list | grep sklearn
sklearn                   0.0
```

### Why are the changes needed?

This will save the installation time.

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

No.

### How was this patch tested?

Pass the GitHub Action PySpark jobs.

Closes #33595 from dongjoon-hyun/SPARK-36345.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 0e65ed5fb9)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-31 07:21:15 +09:00
Andy Grove f9f5656491 [SPARK-35881][SQL] Add support for columnar execution of final query stage in AdaptiveSparkPlanExec
### What changes were proposed in this pull request?

Changes in this PR:

- `AdaptiveSparkPlanExec` has new methods `finalPlanSupportsColumnar` and `doExecuteColumnar` to support adaptive queries where the final query stage produces columnar data.
- `SessionState` now has a new set of injectable rules named `finalQueryStagePrepRules` that can be applied to the final query stage.
- `AdaptiveSparkPlanExec` can now safely be wrapped by either `RowToColumnarExec` or `ColumnarToRowExec`.

A Spark plugin can use the new rules to remove the root `ColumnarToRowExec` transition that is inserted by previous rules and at execution time can call `finalPlanSupportsColumnar` to see if the final query stage is columnar. If the plan is columnar then the plugin can safely call `doExecuteColumnar`. The adaptive plan can be wrapped in either `RowToColumnarExec` or `ColumnarToRowExec` to force a particular output format. There are fast paths in both of these operators to avoid any redundant transitions.

### Why are the changes needed?

Without this change it is necessary to use reflection to get the final physical plan to determine whether it is columnar and to execute it is a columnar plan. `AdaptiveSparkPlanExec` only provides public methods for row-based execution.

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

No.

### How was this patch tested?

I have manually tested this patch with the RAPIDS Accelerator for Apache Spark.

Closes #33140 from andygrove/support-columnar-adaptive.

Authored-by: Andy Grove <andygrove73@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
(cherry picked from commit 0f538402fb)
Signed-off-by: Thomas Graves <tgraves@apache.org>
2021-07-30 15:38:52 -05:00
Takuya UESHIN a4dcda1794 [SPARK-36350][PYTHON] Move some logic related to F.nanvl to DataTypeOps
### What changes were proposed in this pull request?

Move some logic related to `F.nanvl` to `DataTypeOps`.

### Why are the changes needed?

There are several places to branch by `FloatType` or `DoubleType` to use `F.nanvl` but `DataTypeOps` should handle it.

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

No.

### How was this patch tested?

Existing tests.

Closes #33582 from ueshin/issues/SPARK-36350/nan_to_null.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
(cherry picked from commit 895e3f5e2a)
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-07-30 11:20:01 -07:00
Hyukjin Kwon fee87f13d1 [SPARK-36338][PYTHON][SQL] Move distributed-sequence implementation to Scala side
### What changes were proposed in this pull request?

This PR proposes to implement `distributed-sequence` index in Scala side.

### Why are the changes needed?

- Avoid unnecessary (de)serialization
- Keep the nullability in the input DataFrame when `distributed-sequence` is enabled. During the serialization, all fields are being nullable for now (see https://github.com/apache/spark/pull/32775#discussion_r645882104)

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

No to end users since pandas API on Spark is not released yet.

```python
import pyspark.pandas as ps
ps.set_option('compute.default_index_type', 'distributed-sequence')
ps.range(1).spark.print_schema()
```

Before:

```
root
 |-- id: long (nullable = true)
```

After:

```
root
 |-- id: long (nullable = false)
```

### How was this patch tested?

Manually tested, and existing tests should cover them.

Closes #33570 from HyukjinKwon/SPARK-36338.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit c6140d4d0a)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-30 22:29:31 +09:00
Hyukjin Kwon 9cd370894b [SPARK-36254][PYTHON][FOLLOW-UP] Skip mlflow related tests in pandas on Spark
### What changes were proposed in this pull request?

This PR is a partial revert of https://github.com/apache/spark/pull/33567 that keeps the logic to skip mlflow related tests if that's not installed.

### Why are the changes needed?

It's consistent with other libraries, e.g) PyArrow.
It also fixes up the potential dev breakage (see also https://github.com/apache/spark/pull/33567#issuecomment-889841829)

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

No, dev-only.

### How was this patch tested?

This is a partial revert. CI should test it out too.

Closes #33589 from HyukjinKwon/SPARK-36254.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit dd2ca0aee2)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-30 22:28:29 +09:00
Wenchen Fan f6bb75b0bc [SPARK-34952][SQL][FOLLOWUP] Simplify JDBC aggregate pushdown
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/33352 , to simplify the JDBC aggregate pushdown:
1. We should get the schema of the aggregate query by asking the JDBC server, instead of calculating it by ourselves. This can simplify the code a lot, and is also more robust: the data type of SUM may vary in different databases, it's fragile to assume they are always the same as Spark.
2. because of 1, now we can remove the `dataType` property from the public `Sum` expression.

This PR also contains some small improvements:
1. Spark should deduplicate the aggregate expressions before pushing them down.
2. Improve the `toString` of public aggregate expressions to make them more SQL.

### Why are the changes needed?

code and API simplification

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

this API is not released yet.

### How was this patch tested?

existing tests

Closes #33579 from cloud-fan/dsv2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 387a251a68)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-30 00:26:41 -07:00
itholic a9c5b1a5c8 [SPARK-36254][INFRA][PYTHON] Install mlflow in Github Actions CI
### What changes were proposed in this pull request?

This PR proposes adding a Python package, `mlflow` and `sklearn` to enable the MLflow test in pandas API on Spark.

### Why are the changes needed?

To enable the MLflow test in pandas API on Spark.

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

No, it's test-only

### How was this patch tested?

Manually test on local, with `python/run-tests --testnames pyspark.pandas.mlflow`.

Closes #33567 from itholic/SPARK-36254.

Lead-authored-by: itholic <haejoon.lee@databricks.com>
Co-authored-by: Haejoon Lee <44108233+itholic@users.noreply.github.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit abce61f3fd)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-30 00:04:59 -07:00
Angerszhuuuu a96e9e197e [SPARK-34399][SQL][3.2] Add commit duration to SQL tab's graph node
### What changes were proposed in this pull request?
Since we have add log about commit time, I think this useful and we can make user know it directly in SQL tab's UI.

![image](https://user-images.githubusercontent.com/46485123/126647754-dc3ba83a-5391-427c-8a67-e6af46e82290.png)

### Why are the changes needed?
Make user can directly know commit duration.

### Does this PR introduce _any_ user-facing change?
User can see file commit duration in SQL tab's SQL plan graph

### How was this patch tested?
Mannul tested

Closes #33553 from AngersZhuuuu/SPARK-34399-FOLLOWUP.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-30 12:30:20 +08:00
itholic 26b8297fa3 [SPARK-35806][PYTHON][FOLLOW-UP] Mapping the mode argument to pandas in DataFrame.to_csv
### What changes were proposed in this pull request?

This PR is follow-up for https://github.com/apache/spark/pull/33414 to support the more options for `mode` argument for all APIs that has `mode` argument, not only `DataFrame.to_csv`.

### Why are the changes needed?

To keep the usage consistency for the arguments that have same name.

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

More options is available for all APIs that has `mode` argument, same as `DataFrame.to_csv`

### How was this patch tested?

Manually test on local

Closes #33569 from itholic/SPARK-35085-followup.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 94cb2bbbc2)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-30 12:48:36 +09:00
Chao Sun 8c203272de [SPARK-36136][SQL][TESTS] Refactor PruneFileSourcePartitionsSuite etc to a different package
### What changes were proposed in this pull request?

Move both `PruneFileSourcePartitionsSuite` and `PrunePartitionSuiteBase` to the package `org.apache.spark.sql.execution.datasources`. Did a few refactoring to enable this.

### Why are the changes needed?

Currently both `PruneFileSourcePartitionsSuite` and `PrunePartitionSuiteBase` are in package `org.apache.spark.sql.hive.execution` which doesn't look correct as these tests are not specific to Hive. Therefore, it's better to move them into `org.apache.spark.sql.execution.datasources`, the same place where the rule `PruneFileSourcePartitions` is at.

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

No, it's just test refactoring.

### How was this patch tested?

Using existing tests:
```
build/sbt "sql/testOnly *PruneFileSourcePartitionsSuite"
```
and
```
build/sbt "hive/testOnly *PruneHiveTablePartitionsSuite"
```

Closes #33564 from sunchao/SPARK-36136-partitions-suite.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 0ece865ea4)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-29 17:18:33 -07:00
zhuqi-lucas d37f732a68 [SPARK-36344][CORE][SHUFFLE] Fix some typos in ShuffleBlockPusher class
### What changes were proposed in this pull request?
Just to fix some typos in ShuffleBlockPusher class.

### Why are the changes needed?
Fix the typos, make code clear.

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

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

Closes #33575 from zhuqi-lucas/master.

Authored-by: zhuqi-lucas <821684824@qq.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 900b38d5fa)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-30 09:11:25 +09:00
Takuya UESHIN 359dfd7ec2 [SPARK-36333][PYTHON] Reuse isnull where the null check is needed
### What changes were proposed in this pull request?

Reuse `IndexOpsMixin.isnull()` where the null check is needed.

### Why are the changes needed?

There are some places where we can reuse `IndexOpsMixin.isnull()` instead of directly using Spark `Column`.

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

No.

### How was this patch tested?

Existing tests.

Closes #33562 from ueshin/issues/SPARK-36333/reuse_isnull.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
(cherry picked from commit 07ed82be0b)
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-07-29 15:33:25 -07:00
Yuanjian Li e8462a584c [SPARK-36347][SS] Upgrade the RocksDB version to 6.20.3
### What changes were proposed in this pull request?
As the discussion in https://github.com/apache/spark/pull/32928/files#r654049392, after confirming the compatibility, we can use a newer RocksDB version for the state store implementation.

### Why are the changes needed?
For further ARM support and leverage the bug fix for the newer version.

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

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

Closes #33578 from xuanyuanking/SPARK-36347.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
(cherry picked from commit 4cd5fa96d8)
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-07-29 11:09:10 -07:00
Samuel Moseley e42324982d [SPARK-36161][PYTHON] Add type check on dropDuplicates pyspark function
### What changes were proposed in this pull request?
Improve the error message for wrong type when calling dropDuplicates in pyspark.

### Why are the changes needed?
The current error message is cryptic and can be unclear to less experienced users.

### Does this PR introduce _any_ user-facing change?
Yes, it adds a type error for when a user gives the wrong type to dropDuplicates

### How was this patch tested?
There is currently no testing for error messages in pyspark dataframe functions

Closes #33364 from sammyjmoseley/sm/add-type-checking-for-drop-duplicates.

Lead-authored-by: Samuel Moseley <smoseley@palantir.com>
Co-authored-by: Sammy Moseley <moseley.sammy@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit a07df1acc6)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-29 19:11:57 +09:00
Kousuke Saruta d247a6cd1a [SPARK-36323][SQL] Support ANSI interval literals for TimeWindow
### What changes were proposed in this pull request?

This PR proposes to support ANSI interval literals for `TimeWindow`.

### Why are the changes needed?

Watermark also supports ANSI interval literals so it's great to support for `TimeWindow`.

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

No.

### How was this patch tested?

New test.

Closes #33551 from sarutak/window-interval.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit db18866742)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-29 08:52:10 +03:00
Cheng Su 6d188cbb08 [SPARK-36272][SQL][TEST] Change shuffled hash join metrics test to check relative value of build size
### What changes were proposed in this pull request?

This is a follow up of https://github.com/apache/spark/pull/33447, where the unit test is disabled, due to failure after memory setting changed. I found the root cause is after https://github.com/apache/spark/pull/33447, in unit test, Spark memory page byte size is changed from `67108864` to `33554432` [1]. So the shuffled hash join build size is also changed accordingly due to [memory page byte size change](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala#L457). Previously the unit test is checking the exact value of build size, so it no longer works. Here we change the unit test to verify the relative value of build size, and it should work.

[1]: I printed out the memory page byte size explicitly in unit test - `org.apache.spark.SparkException: chengsu pageSizeBytes: 33554432!` in https://github.com/c21/spark/runs/3186680616?check_suite_focus=true .

### Why are the changes needed?

Make previously disabled unit test work.

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

No.

### How was this patch tested?

Changed unit test itself.

Closes #33494 from c21/test.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 6a8dd3229a)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-29 11:14:48 +09:00
Linhong Liu fa521c1506 [SPARK-36286][SQL] Block some invalid datetime string
### What changes were proposed in this pull request?
In PR #32959, we found some weird datetime strings that can be parsed. ([details](https://github.com/apache/spark/pull/32959#discussion_r665015489))
This PR blocks the invalid datetime string.

### Why are the changes needed?
bug fix

### Does this PR introduce _any_ user-facing change?
Yes, below strings will have different results when cast to datetime.
```sql
select cast('12::' as timestamp); -- Before: 2021-07-07 12:00:00, After: NULL
select cast('T' as timestamp); -- Before: 2021-07-07 00:00:00, After: NULL
```

### How was this patch tested?
some new test cases

Closes #33490 from linhongliu-db/SPARK-35780-block-invalid-format.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit ed0e351f05)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-29 09:17:01 +08:00
Xinrong Meng 999cf81653 [SPARK-36190][PYTHON] Improve the rest of DataTypeOps tests by avoiding joins
### What changes were proposed in this pull request?
Improve the rest of DataTypeOps tests by avoiding joins.

### Why are the changes needed?
bool, string, numeric DataTypeOps tests have been improved by avoiding joins.
We should improve the rest of the DataTypeOps tests in the same way.

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

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

Closes #33546 from xinrong-databricks/test_no_join.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
(cherry picked from commit 9c5cb99d6e)
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-07-28 15:53:51 -07:00
Venki Korukanti c236101d4c [SPARK-36236][SS] Additional metrics for RocksDB based state store implementation
### What changes were proposed in this pull request?

Proposing adding new metrics to `customMetrics` under the `stateOperators` in `StreamingQueryProgress` event These metrics help have better visibility into the RocksDB based state store in streaming jobs. For full details of metrics, refer to https://issues.apache.org/jira/browse/SPARK-36236.

### Why are the changes needed?

Current metrics available for the RockDB state store, do not provide observability into many operations such as how much time is spent by the RocksDB in compaction and what is the cache hit ratio. These metrics help compare performance differences in state store operations between slow and fast microbatches .

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

No

### How was this patch tested?

Unittests

Closes #33455 from vkorukanti/rocksdb-metrics.

Authored-by: Venki Korukanti <venki.korukanti@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit eb4d1c0332)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-28 12:51:56 -07:00
Xinrong Meng bdf1570911 [SPARK-36143][PYTHON] Adjust astype of fractional Series with missing values to follow pandas
### What changes were proposed in this pull request?
Adjust `astype` of fractional Series with missing values to follow pandas.

Non-goal: Adjust the issue of `astype` of Decimal Series with missing values to follow pandas.

### Why are the changes needed?
`astype` of fractional Series with missing values doesn't behave the same as pandas, for example, float Series returns itself when `astype` integer, while a ValueError is raised in pandas.

We ought to follow pandas.

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

From:
```py
>>> import numpy as np
>>> import pyspark.pandas as ps
>>> psser = ps.Series([1, 2, np.nan])
>>> psser.astype(int)
0    1.0
1    2.0
2    NaN
dtype: float64

```

To:
```py
>>> import numpy as np
>>> import pyspark.pandas as ps
>>> psser = ps.Series([1, 2, np.nan])
>>> psser.astype(int)
Traceback (most recent call last):
...
ValueError: Cannot convert fractions with missing values to integer

```

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

Closes #33466 from xinrong-databricks/extension_astype.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
(cherry picked from commit 01213095e2)
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-07-28 11:26:58 -07:00
dgd-contributor c5b0cb2d94 [SPARK-36229][SQL] conv() inconsistently handles invalid strings with more than 64 invalid characters and return wrong value on overflow
### What changes were proposed in this pull request?
1/ conv() have inconsistency in behavior where the returned value is different above the 64 char threshold.

```
scala> spark.sql("select conv(repeat('?', 64), 10, 16)").show
+---------------------------+
|conv(repeat(?, 64), 10, 16)|
+---------------------------+
|                          0|
+---------------------------+

scala> spark.sql("select conv(repeat('?', 65), 10, 16)").show // which should be 0
+---------------------------+
|conv(repeat(?, 65), 10, 16)|
+---------------------------+
|           FFFFFFFFFFFFFFFF|
+---------------------------+

scala> spark.sql("select conv(repeat('?', 65), 10, -16)").show // which should be 0
+----------------------------+
|conv(repeat(?, 65), 10, -16)|
+----------------------------+
|                          -1|
+----------------------------+

scala> spark.sql("select conv(repeat('?', 64), 10, -16)").show
+----------------------------+
|conv(repeat(?, 64), 10, -16)|
+----------------------------+
|                           0|
+----------------------------+
```

2/ conv should return result equal to max unsigned long value in base toBase when there is overflow

```
scala> spark.sql(select conv('aaaaaaa0aaaaaaa0a', 16, 10)).show // which should be 18446744073709551615

+-------------------------------+
|conv(aaaaaaa0aaaaaaa0a, 16, 10)|
+-------------------------------+
|           12297828695278266890|
+-------------------------------+
```

### Why are the changes needed?
Bug fix, this pull request aim to make conv function behave similarly with the behavior of conv function from MySQL database
### Does this PR introduce _any_ user-facing change?
change in result of conv() function
### How was this patch tested?
add test

Closes #33459 from dgd-contributor/SPARK-36229_convInconsistencyBehaviorWithMoreThan64Characters.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit e1c50ff779)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-29 00:19:19 +08:00
Takuya UESHIN c9af94ecb4 [SPARK-36320][PYTHON] Fix Series/Index.copy() to drop extra columns
### What changes were proposed in this pull request?

Fix `Series`/`Index.copy()` to drop extra columns.

### Why are the changes needed?

Currently `Series`/`Index.copy()` keeps the copy of the anchor DataFrame which holds unnecessary columns.
We can drop those when `Series`/`Index.copy()`.

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

No.

### How was this patch tested?

Existing tests.

Closes #33549 from ueshin/issues/SPARK-36320/index_ops_copy.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 3c76a924ce)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-28 18:40:03 +09:00
Linhong Liu 3dc74e7aa6 [SPARK-36270][BUILD][FOLLOWUP] Fix typo in the sbt memory setting
### What changes were proposed in this pull request?
remove the `$` in the `-Xms$256m` memory options

### Why are the changes needed?
fix typo

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

### How was this patch tested?
not needed

Closes #33557 from linhongliu-db/minor-fix.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit aff1826331)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-28 17:18:07 +09:00
Angerszhuuuu b58170b192 [SPARK-36312][SQL][FOLLOWUP] Add back ParquetSchemaConverter.checkFieldNames
### What changes were proposed in this pull request?
Add back ParquetSchemaConverter.checkFieldNames()

### Why are the changes needed?
Fix code

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

### How was this patch tested?

Closes #33552 from AngersZhuuuu/SPARK-36312-FOLLOWUP.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit f086c17b8e)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-28 14:38:36 +08:00
Angerszhuuuu 2f4f7936fd [SPARK-33865][SPARK-36202][SQL] When HiveDDL, we need check avro schema too
### What changes were proposed in this pull request?
Unify schema check code of FileFormat and check avro schema filed name when CREATE TABLE DDL too

### Why are the changes needed?
Refactor code

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

### How was this patch tested?
Not need

Closes #33441 from AngersZhuuuu/SPARK-36202.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 86f44578e5)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-28 14:04:37 +08:00
Terry Kim cd6b303d0f [SPARK-36006][SQL] Migrate ALTER TABLE ... ADD/REPLACE COLUMNS commands to use UnresolvedTable to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate the following `ALTER TABLE ... ADD/REPLACE COLUMNS` commands to use `UnresolvedTable` as a `child` to resolve the table identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).

### Why are the changes needed?

This is a part of effort to make the relation lookup behavior consistent: [SPARK-29900](https://issues.apache.org/jira/browse/SPARK-29900).

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

After this PR, the above `ALTER TABLE ... ADD/REPLACE COLUMNS` commands will have a consistent resolution behavior.

### How was this patch tested?

Updated existing tests.

Closes #33200 from imback82/alter_add_cols.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 809b88a162)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-28 14:00:44 +08:00
Gengliang Wang fa0c7f487b [SPARK-34399][DOCS][FOLLOWUP] Add docs for the new metrics of task/job commit time
### What changes were proposed in this pull request?

This is follow-up of https://github.com/apache/spark/pull/31522.
It adds docs for the new metrics of task/job commit time

### Why are the changes needed?

So that users can understand the metrics better and know that the new metrics are only for file table writes.

### 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/127198210-2ab201d3-5fca-4065-ace6-0b930390380f.png)

Closes #33542 from gengliangwang/addDocForMetrics.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit c9a7ff3f36)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-28 13:54:48 +08:00
Angerszhuuuu 3c441135bb [SPARK-36312][SQL] ParquetWriterSupport.setSchema should check inner field
### What changes were proposed in this pull request?
Last pr only support add inner field check for hive ddl, this pr add check for parquet data source write API.

### Why are the changes needed?
Failed earlier

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

### How was this patch tested?
Added Ut

Without this UI it failed as
```
[info] - SPARK-36312: ParquetWriteSupport should check inner field *** FAILED *** (8 seconds, 29 milliseconds)
[info]   Expected exception org.apache.spark.sql.AnalysisException to be thrown, but org.apache.spark.SparkException was thrown (HiveDDLSuite.scala:3035)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
[info]   at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
[info]   at org.scalatest.funsuite.AnyFunSuite.newAssertionFailedException(AnyFunSuite.scala:1563)
[info]   at org.scalatest.Assertions.intercept(Assertions.scala:756)
[info]   at org.scalatest.Assertions.intercept$(Assertions.scala:746)
[info]   at org.scalatest.funsuite.AnyFunSuite.intercept(AnyFunSuite.scala:1563)
[info]   at org.apache.spark.sql.hive.execution.HiveDDLSuite.$anonfun$new$396(HiveDDLSuite.scala:3035)
[info]   at org.apache.spark.sql.hive.execution.HiveDDLSuite.$anonfun$new$396$adapted(HiveDDLSuite.scala:3034)
[info]   at org.apache.spark.sql.catalyst.plans.SQLHelper.withTempPath(SQLHelper.scala:69)
[info]   at org.apache.spark.sql.catalyst.plans.SQLHelper.withTempPath$(SQLHelper.scala:66)
[info]   at org.apache.spark.sql.QueryTest.withTempPath(QueryTest.scala:34)
[info]   at org.apache.spark.sql.hive.execution.HiveDDLSuite.$anonfun$new$395(HiveDDLSuite.scala:3034)
[info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
[info]   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1468)
[info]   at org.apache.spark.sql.test.SQLTestUtilsBase.withView(SQLTestUtils.scala:316)
[info]   at org.apache.spark.sql.test.SQLTestUtilsBase.withView$(SQLTestUtils.scala:314)
[info]   at org.apache.spark.sql.hive.execution.HiveDDLSuite.withView(HiveDDLSuite.scala:396)
[info]   at org.apache.spark.sql.hive.execution.HiveDDLSuite.$anonfun$new$394(HiveDDLSuite.scala:3032)
[info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
[info]   at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
[info]   at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
[info]   at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:22)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:20)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
[info]   at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
[info]   at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
[info]   at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
[info]   at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
[info]   at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
[info]   at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
[info]   at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
[info]   at scala.collection.immutable.List.foreach(List.scala:431)
[info]   at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
[info]   at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396)
[info]   at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:269)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:268)
[info]   at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1563)
[info]   at org.scalatest.Suite.run(Suite.scala:1112)
[info]   at org.scalatest.Suite.run$(Suite.scala:1094)
[info]   at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1563)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:273)
[info]   at org.scalatest.SuperEngine.runImpl(Engine.scala:535)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:273)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:272)
[info]   at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:62)
[info]   at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
[info]   at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
[info]   at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
[info]   at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:62)
[info]   at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:318)
[info]   at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:513)
[info]   at sbt.ForkMain$Run.lambda$runTest$1(ForkMain.java:413)
[info]   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
[info]   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
[info]   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[info]   at java.lang.Thread.run(Thread.java:748)
[info]   Cause: org.apache.spark.SparkException: Job aborted.
[info]   at org.apache.spark.sql.errors.QueryExecutionErrors$.jobAbortedError(QueryExecutionErrors.scala:496)
[info]   at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:251)
[info]   at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:186)
[info]   at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:113)
[info]   at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:111)
[info]   at org.apache.spark.sql.execution.command.DataWritingCommandExec.executeCollect(commands.scala:125)
[info]   at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:97)
[info]   at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
[info]   at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
[info]   at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
[info]   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
[info]   at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
[info]   at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:97)
[info]   at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:93)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:481)
[info]   at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:82)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:481)
[info]   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
[info]   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
[info]   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
[info]   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
[info]   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:457)
[info]   at org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:93)
[info]   at org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:80)
[info]   at org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:78)
[info]   at org.apache.spark.sql.execution.QueryExecution.assertCommandExecuted(QueryExecution.scala:115)
[info]   at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:848)
[info]   at org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:382)
[info]   at org.apache.spark.sql.DataFrameWriter.saveInternal(DataFrameWriter.scala:355)
[info]   at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:239)
[info]   at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:781)
[in
```

Closes #33531 from AngersZhuuuu/SPARK-36312.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 59e0c25376)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-28 13:52:40 +08:00
Eugene Koifman c59e54fe0e [SPARK-35639][SQL] Add metrics about coalesced partitions to AQEShuffleRead in AQE
### What changes were proposed in this pull request?

AQEShuffleReadExec already reports "number of skewed partitions" and "number of skewed partition splits".
It would be useful to also report "number of coalesced partitions" and for ShuffleExchange to report "number of partitions"
This way it's clear what happened on the map side and on the reduce side.

![Metrics](https://user-images.githubusercontent.com/4297661/126729820-cf01b3fa-7bc4-44a5-8098-91689766a68a.png)

### Why are the changes needed?

Improves usability

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

Yes, it now provides more information about `AQEShuffleReadExec` operator behavior in the metrics system.

### How was this patch tested?

Existing tests

Closes #32776 from ekoifman/PRISM-91635-customshufflereader-sql-metrics.

Authored-by: Eugene Koifman <eugene.koifman@workday.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 41a16ebf11)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-28 13:50:04 +08:00
allisonwang-db 993ffafc3e [SPARK-36275][SQL] ResolveAggregateFunctions should works with nested fields
### What changes were proposed in this pull request?
This PR fixes an issue in `ResolveAggregateFunctions` where non-aggregated nested fields in ORDER BY and HAVING are not resolved correctly. This is because nested fields are resolved as aliases that fail to be semantically equal to any grouping/aggregate expressions.

### Why are the changes needed?
To fix an analyzer issue.

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

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

Closes #33498 from allisonwang-db/spark-36275-resolve-agg-func.

Authored-by: allisonwang-db <allison.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 23a6ffa5dc)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-28 13:35:35 +08:00
allisonwang-db aea36aa977 [SPARK-36028][SQL][3.2] Allow Project to host outer references in scalar subqueries
This PR cherry picks https://github.com/apache/spark/pull/33235 to branch-3.2 to fix test failures introduced by https://github.com/apache/spark/pull/33284.

### What changes were proposed in this pull request?
This PR allows the `Project` node to host outer references in scalar subqueries when `decorrelateInnerQuery` is enabled. It is already supported by the new decorrelation framework and the `RewriteCorrelatedScalarSubquery` rule.

Note currently by default all correlated subqueries will be decorrelated, which is not necessarily the most optimal approach. Consider `SELECT (SELECT c1) FROM t`. This should be optimized as `SELECT c1 FROM t` instead of rewriting it as a left outer join. This will be done in a separate PR to optimize correlated scalar/lateral subqueries with OneRowRelation.

### Why are the changes needed?
To allow more types of correlated scalar subqueries.

### Does this PR introduce _any_ user-facing change?
Yes. This PR allows outer query column references in the SELECT cluase of a correlated scalar subquery. For example:
```sql
SELECT (SELECT c1) FROM t;
```
Before this change:
```
org.apache.spark.sql.AnalysisException: Expressions referencing the outer query are not supported
outside of WHERE/HAVING clauses
```

After this change:
```
+------------------+
|scalarsubquery(c1)|
+------------------+
|0                 |
|1                 |
+------------------+
```

### How was this patch tested?
Added unit tests and SQL tests.

(cherry picked from commit ca348e50a4)
Signed-off-by: allisonwang-db <allison.wangdatabricks.com>

Closes #33527 from allisonwang-db/spark-36028-3.2.

Authored-by: allisonwang-db <allison.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-28 12:54:15 +08:00
Huaxin Gao 33ef52e2c0 [SPARK-34952][SQL][FOLLOW-UP] DSv2 aggregate push down follow-up
### What changes were proposed in this pull request?
update java doc, JDBC data source doc, address follow up comments

### Why are the changes needed?
update doc and address follow up comments

### Does this PR introduce _any_ user-facing change?
Yes, add the new JDBC option `pushDownAggregate` in JDBC data source doc.

### How was this patch tested?
manually checked

Closes #33526 from huaxingao/aggPD_followup.

Authored-by: Huaxin Gao <huaxin_gao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit c8dd97d456)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-28 12:52:58 +08:00
Max Gekk 56f1ee4b06 [SPARK-36318][SQL][DOCS] Update docs about mapping of ANSI interval types to Java/Scala/SQL types
### What changes were proposed in this pull request?
1. Update the tables at https://spark.apache.org/docs/latest/sql-ref-datatypes.html about mapping ANSI interval types to Java/Scala/SQL types.
2. Remove `CalendarIntervalType` from the table of mapping Catalyst types to SQL types.

<img width="1028" alt="Screenshot 2021-07-27 at 20 52 57" src="https://user-images.githubusercontent.com/1580697/127204790-7ccb9c64-daf2-427d-963e-b7367aaa3439.png">
<img width="1017" alt="Screenshot 2021-07-27 at 20 53 22" src="https://user-images.githubusercontent.com/1580697/127204806-a0a51950-3c2d-4198-8a22-0f6614bb1487.png">

### Why are the changes needed?
To inform users which types from language APIs should be used as ANSI interval types.

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

### How was this patch tested?
Manually checking by building the docs:
```
$ SKIP_RDOC=1 SKIP_API=1 SKIP_PYTHONDOC=1 bundle exec jekyll build
```

Closes #33543 from MaxGekk/doc-interval-type-lang-api.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
(cherry picked from commit 1614d00417)
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-07-28 13:42:53 +09:00
Jungtaek Lim 16c6009957 [SPARK-36314][SS] Update Sessionization examples to use native support of session window
### What changes were proposed in this pull request?

This PR proposes to update Sessionization examples to use native support of session window. It also adds the example for PySpark as native support of session window is available to PySpark as well.

### Why are the changes needed?

We should guide the simplest way to achieve the same workload. I'll provide another example for cases we can't do with native support of session window.

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

No.

### How was this patch tested?

Manually tested.

Closes #33548 from HeartSaVioR/SPARK-36314.

Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 1fafa8e191)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-27 20:10:13 -07:00
Takuya UESHIN 0e9e737a84 [SPARK-36310][PYTHON] Fix IndexOpsMixin.hasnans to use isnull().any()
### What changes were proposed in this pull request?

Fix `IndexOpsMixin.hasnans` to use `IndexOpsMixin.isnull().any()`.

### Why are the changes needed?

`IndexOpsMixin.hasnans` has a potential issue to cause `a window function inside an aggregate function` error.
Also it returns a wrong value when the `Series`/`Index` is empty.

```py
>>> ps.Series([]).hasnans
None
```

whereas:

```py
>>> pd.Series([]).hasnans
False
```

`IndexOpsMixin.any()` is safe for both cases.

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

`IndexOpsMixin.hasnans` will return `False` when empty.

### How was this patch tested?

Added some tests.

Closes #33547 from ueshin/issues/SPARK-36310/hasnan.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit bcc595c112)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-28 09:21:22 +09:00
Gengliang Wang ee3bd71c92 [SPARK-34249][DOCS] Add documentation for ANSI implicit cast rules
### What changes were proposed in this pull request?

Add documentation for the ANSI implicit cast rules which are introduced from https://github.com/apache/spark/pull/31349

### Why are the changes needed?

Better documentation.

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

No
### How was this patch tested?

Build and preview in local:
![image](https://user-images.githubusercontent.com/1097932/127149039-f0cc4766-8eca-4061-bc35-c8e67f009544.png)
![image](https://user-images.githubusercontent.com/1097932/127149072-1b65ef56-65ff-4327-9a5e-450d44719073.png)

![image](https://user-images.githubusercontent.com/1097932/127033375-b4536854-ca72-42fa-8ea9-dde158264aa5.png)
![image](https://user-images.githubusercontent.com/1097932/126950445-435ba521-92b8-44d1-8f2c-250b9efb4b98.png)
![image](https://user-images.githubusercontent.com/1097932/126950495-9aa4e960-60cd-4b20-88d9-b697ff57a7f7.png)

Closes #33516 from gengliangwang/addDoc.

Lead-authored-by: Gengliang Wang <gengliang@apache.org>
Co-authored-by: Serge Rielau <serge@rielau.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit df98d5b5f1)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-27 20:49:05 +08:00
Max Gekk 3d86128eae [SPARK-34619][SQL][DOCS][3.2] Describe ANSI interval types at the Data types page of the SQL reference
### What changes were proposed in this pull request?
In the PR, I propose to update the page https://spark.apache.org/docs/latest/sql-ref-datatypes.html and add information about the year-month and day-time interval types introduced by SPARK-27790.

<img width="932" alt="Screenshot 2021-07-27 at 10 38 23" src="https://user-images.githubusercontent.com/1580697/127115289-e633ca3a-2c18-49a0-a7c0-22421ae5c363.png">

### Why are the changes needed?
To inform users about new ANSI interval types, and improve UX with Spark SQL.

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

### How was this patch tested?
Should be tested by a GitHub action.

Authored-by: Max Gekk <max.gekkgmail.com>
Signed-off-by: Kousuke Saruta <sarutakoss.nttdata.com>
(cherry picked from commit f4837961a9)
Signed-off-by: Kousuke Saruta <sarutakoss.nttdata.com>

Closes #33539 from sarutak/backport-SPARK-34619-3.2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-27 15:17:03 +03:00
Liang-Chi Hsieh dcd37f9639 Revert "[SPARK-36136][SQL][TESTS] Refactor PruneFileSourcePartitionsSuite etc to a different package"
This reverts commit 634f96dde4.

Closes #33533 from viirya/revert-SPARK-36136.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 22ac98dcbf)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-27 19:11:51 +09:00