Commit graph

31287 commits

Author SHA1 Message Date
Gengliang Wang 7ac0a2c37b [SPARK-36827][CORE] Improve the perf and memory usage of cleaning up stage UI data
### What changes were proposed in this pull request?

Improve the perf and memory usage of cleaning up stage UI data. The new code make copy of the essential fields(stage id, attempt id, completion time) to an array and determine which stage data and `RDDOperationGraphWrapper` needs to be clean based on it
### Why are the changes needed?

Fix the memory usage issue described in https://issues.apache.org/jira/browse/SPARK-36827

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

No

### How was this patch tested?

Add new unit test for the InMemoryStore.
Also, run a simple benchmark with
```
    val testConf = conf.clone()
      .set(MAX_RETAINED_STAGES, 1000)

    val listener = new AppStatusListener(store, testConf, true)
    val stages = (1 to 5000).map { i =>
      val s = new StageInfo(i, 0, s"stage$i", 4, Nil, Nil, "details1",
        resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)
      s.submissionTime = Some(i.toLong)
      s
    }
    listener.onJobStart(SparkListenerJobStart(4, time, Nil, null))
    val start = System.nanoTime()
    stages.foreach { s =>
      time +=1
      s.submissionTime = Some(time)
      listener.onStageSubmitted(SparkListenerStageSubmitted(s, new Properties()))
      s.completionTime = Some(time)
      listener.onStageCompleted(SparkListenerStageCompleted(s))
    }
    println(System.nanoTime() - start)
```

Before changes:
InMemoryStore: 1.2s

After changes:
InMemoryStore: 0.23s

Closes #34092 from gengliangwang/cleanStage.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-09-24 17:24:18 +08:00
allisonwang-db 52833018be [SPARK-36656][SQL][TEST] CollapseProject should not collapse correlated scalar subqueries
### What changes were proposed in this pull request?
The optimizer rule `CollapseProject` used to inline expressions with correlated scalar subqueries, which can lead to inefficient and invalid plans. This issue was fixed in #33958. This PR adds an additional test to verify the behavior.

### Why are the changes needed?
To make sure CollapseProject works with correlated scalar subqueries.

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

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

Closes #33903 from allisonwang-db/spark-36656-collapse-project-subquery.

Authored-by: allisonwang-db <allison.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-24 16:52:11 +08:00
Angerszhuuuu 64f4bf47af [SPARK-36792][SQL] InSet should handle NaN
### What changes were proposed in this pull request?
InSet should handle NaN
```
InSet(Literal(Double.NaN), Set(Double.NaN, 1d)) should return true, but return false.
```
### Why are the changes needed?
InSet should handle NaN

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

### How was this patch tested?
Added UT

Closes #34033 from AngersZhuuuu/SPARK-36792.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-24 16:19:36 +08:00
Max Gekk c77c0d41e1 [SPARK-36825][SQL] Read/write dataframes with ANSI intervals from/to parquet files
### What changes were proposed in this pull request?
Allow saving and loading of ANSI intervals - `YearMonthIntervalType` and `DayTimeIntervalType` to/from the Parquet datasource. After the changes, Spark saves ANSI intervals as primitive physical Parquet types:
- year-month intervals as `INT32`
- day-time intervals as `INT64`

w/o any modifications. To load the values as intervals back, Spark puts the info about interval types to the extra key `org.apache.spark.sql.parquet.row.metadata`:
```
$ java -jar parquet-tools-1.12.0.jar meta ./part-...-c000.snappy.parquet

creator:     parquet-mr version 1.12.1 (build 2a5c06c58fa987f85aa22170be14d927d5ff6e7d)
extra:       org.apache.spark.version = 3.3.0
extra:       org.apache.spark.sql.parquet.row.metadata = {"type":"struct","fields":[...,{"name":"i","type":"interval year to month","nullable":false,"metadata":{}}]}

file schema: spark_schema
--------------------------------------------------------------------------------
...
i:           REQUIRED INT32 R:0 D:0
```

**Note:** The given PR focus on support of ANSI intervals in the Parquet datasource via write or read as a column in `Dataset`.

### Why are the changes needed?
To improve user experience with Spark SQL. At the moment, users can make ANSI intervals "inside" Spark or parallelize Java collections of `Period`/`Duration` objects but cannot save the intervals to any built-in datasources. After the changes, users can save datasets/dataframes with year-month/day-time intervals to load them back later by Apache Spark.

For example:
```scala
scala> sql("select date'today' - date'2021-01-01' as diff").write.parquet("/Users/maximgekk/tmp/parquet_interval")

scala> val readback = spark.read.parquet("/Users/maximgekk/tmp/parquet_interval")
readback: org.apache.spark.sql.DataFrame = [diff: interval day]

scala> readback.printSchema
root
 |-- diff: interval day (nullable = true)

scala> readback.show
+------------------+
|              diff|
+------------------+
|INTERVAL '264' DAY|
+------------------+
```

### Does this PR introduce _any_ user-facing change?
In some sense, yes. Before the changes, users get an error while saving of ANSI intervals as dataframe columns to parquet files but the operation should complete successfully after the changes.

### How was this patch tested?
1. By running the existing test suites:
```
$ build/sbt "test:testOnly *ParquetFileFormatV2Suite"
$ build/sbt "test:testOnly *FileBasedDataSourceSuite"
$ build/sbt "sql/test:testOnly *JsonV2Suite"
```
2. Added new tests:
```
$ build/sbt "sql/test:testOnly *ParquetIOSuite"
$ build/sbt "sql/test:testOnly *ParquetSchemaSuite"
```

Closes #34057 from MaxGekk/ansi-interval-save-parquet.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-09-24 09:55:11 +03:00
Wenchen Fan 1b2bb38ecd [SPARK-33832][SQL] Force skew join code simplification and improvement
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/32816 to simplify and improve the code:
1. Add a `SkewJoinChildWrapper` to wrap the skew join children, so that `EnsureRequirements` rule will skip them and save time
2. Remove `SkewJoinAwareCost` and keep using `SimpleCost`. We can put `numSkews` in the first 32 bits.

### Why are the changes needed?

code simplification and improvement

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

No

### How was this patch tested?

existing tests

Closes #34080 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-24 13:13:35 +08:00
Chao Sun ed88e610f0 [SPARK-36835][BUILD] Enable createDependencyReducedPom for Maven shaded plugin
### What changes were proposed in this pull request?

Enable `createDependencyReducedPom` for Spark's Maven shaded plugin so that the effective pom won't contain those shaded artifacts such as `org.eclipse.jetty`

### Why are the changes needed?

At the moment, the effective pom leaks transitive dependencies to downstream apps for those shaded artifacts, which potentially will cause issues.

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

No

### How was this patch tested?

I manually tested and the `core/dependency-reduced-pom.xml` no longer contains dependencies such as `jetty-XX`.

Closes #34085 from sunchao/SPARK-36835.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-09-24 10:16:30 +08:00
Peter Toth c2c4a48c78 Revert "[SPARK-35672][CORE][YARN] Pass user classpath entries to executors using config instead of command line"
### What changes were proposed in this pull request?
This reverts commit 866df69c62.

### Why are the changes needed?
After the change environment variables were not substituted in user classpath entries. Please find an example on SPARK-35672.

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

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

Closes #34082 from peter-toth/SPARK-35672-revert.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-24 10:35:39 +09:00
Liang Zhang ef7441bb42 [SPARK-36642][SQL] Add df.withMetadata pyspark API
This PR adds the pyspark API `df.withMetadata(columnName, metadata)`. The scala API is added in this PR https://github.com/apache/spark/pull/33853.

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

To make it easy to use/modify the semantic annotation, we want to have a shorter API to update the metadata in a dataframe. Currently we have `df.withColumn("col1", col("col1").alias("col1", metadata=metadata))` to update the metadata without changing the column name, and this is too verbose. We want to have a syntax suger API `df.withMetadata("col1", metadata=metadata)` to achieve the same functionality.

### Why are the changes needed?

A bit of background for the frequency of the update: We are working on inferring the semantic data types and use them in AutoML and store the semantic annotation in the metadata. So in many cases, we will suggest the user update the metadata to correct the wrong inference or add the annotation for weak inference.

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

Yes.
A syntax suger API `df.withMetadata("col1", metadata=metadata)` to achieve the same functionality as`df.withColumn("col1", col("col1").alias("col1", metadata=metadata))`.

### How was this patch tested?

doctest.

Closes #34021 from liangz1/withMetadataPython.

Authored-by: Liang Zhang <liang.zhang@databricks.com>
Signed-off-by: Weichen Xu <weichen.xu@databricks.com>
2021-09-24 08:30:18 +08:00
Kousuke Saruta 0b65daad2a [SPARK-36760][DOCS][FOLLOWUP] Fix wrong JavaDoc style
### What changes were proposed in this pull request?

This PR fixes a JavaDoc style error introduced in SPARK-36760 (#34073).
Due to this error, build on GA fails and the following error message appears.
```
[error]  * internal -> external data conversion.
```

### Why are the changes needed?

To recover GA.

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

No.

### How was this patch tested?

Should be done by GA itself.

Closes #34078 from sarutak/fix-doc-error.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-09-24 02:18:53 +09:00
Huaxin Gao 6e815da8ea [SPARK-36760][SQL][FOLLOWUP] Add interface SupportsPushDownV2Filters
### What changes were proposed in this pull request?
update java doc...

### Why are the changes needed?
to highlight the difference between this new interface `SupportsPushDownV2Filters` and the old one `SupportsPushDownFilters`

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

### How was this patch tested?
Test not needed

Closes #34073 from huaxingao/followup.

Authored-by: Huaxin Gao <huaxin_gao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-23 23:48:02 +08:00
yi.wu 9d8ac7c8e9 [SPARK-36782][CORE][FOLLOW-UP] Only handle shuffle block in separate thread pool
### What changes were proposed in this pull request?

This's a follow-up of https://github.com/apache/spark/pull/34043. This PR proposes to only handle shuffle blocks in the separate thread pool and leave other blocks the same behavior as it is.

### Why are the changes needed?

To avoid any potential overhead.

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

No.

### How was this patch tested?

Pass existing tests.

Closes #34076 from Ngone51/spark-36782-follow-up.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-09-23 16:29:54 +08:00
Michael Chen 6d7ab7b52b [SPARK-36795][SQL] Explain Formatted has Duplicate Node IDs
### What changes were proposed in this pull request?

Fixed explain formatted mode so it doesn't have duplicate node IDs when InMemoryRelation is present in query plan.

### Why are the changes needed?

Having duplicated node IDs in the plan makes it confusing.

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

Yes, explain formatted string will change.
Notice how `ColumnarToRow` and `InMemoryRelation` have node id of 2.
Before changes =>
```
== Physical Plan ==
AdaptiveSparkPlan (14)
+- == Final Plan ==
   * BroadcastHashJoin Inner BuildLeft (9)
   :- BroadcastQueryStage (5)
   :  +- BroadcastExchange (4)
   :     +- * Filter (3)
   :        +- * ColumnarToRow (2)
   :           +- InMemoryTableScan (1)
   :                 +- InMemoryRelation (2)
   :                       +- * ColumnarToRow (4)
   :                          +- Scan parquet default.t1 (3)
   +- * Filter (8)
      +- * ColumnarToRow (7)
         +- Scan parquet default.t2 (6)
+- == Initial Plan ==
   BroadcastHashJoin Inner BuildLeft (13)
   :- BroadcastExchange (11)
   :  +- Filter (10)
   :     +- InMemoryTableScan (1)
   :           +- InMemoryRelation (2)
   :                 +- * ColumnarToRow (4)
   :                    +- Scan parquet default.t1 (3)
   +- Filter (12)
      +- Scan parquet default.t2 (6)

(1) InMemoryTableScan
Output [1]: [k#x]
Arguments: [k#x], [isnotnull(k#x)]

(2) InMemoryRelation
Arguments: [k#x], CachedRDDBuilder(org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer401788d5,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) ColumnarToRow
+- FileScan parquet default.t1[k#x] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/Users/mike.chen/code/apacheSpark/spark/spark-warehouse/org.apach..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<k:int>
,None)

(3) Scan parquet default.t1
Output [1]: [k#x]
Batched: true
Location: InMemoryFileIndex [file:/Users/mike.chen/code/apacheSpark/spark/spark-warehouse/org.apache.spark.sql.ExplainSuiteAE/t1]
ReadSchema: struct<k:int>

(4) ColumnarToRow [codegen id : 1]
Input [1]: [k#x]

(5) BroadcastQueryStage
Output [1]: [k#x]
Arguments: 0

(6) Scan parquet default.t2
Output [1]: [key#x]
Batched: true
Location: InMemoryFileIndex [file:/Users/mike.chen/code/apacheSpark/spark/spark-warehouse/org.apache.spark.sql.ExplainSuiteAE/t2]
PushedFilters: [IsNotNull(key)]
ReadSchema: struct<key:int>

(7) ColumnarToRow
Input [1]: [key#x]

(8) Filter
Input [1]: [key#x]
Condition : isnotnull(key#x)

(9) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [k#x]
Right keys [1]: [key#x]
Join condition: None

(10) Filter
Input [1]: [k#x]
Condition : isnotnull(k#x)

(11) BroadcastExchange
Input [1]: [k#x]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#x]

(12) Filter
Input [1]: [key#x]
Condition : isnotnull(key#x)

(13) BroadcastHashJoin
Left keys [1]: [k#x]
Right keys [1]: [key#x]
Join condition: None

(14) AdaptiveSparkPlan
Output [2]: [k#x, key#x]
Arguments: isFinalPlan=true
```

After Changes =>
```
== Physical Plan ==
AdaptiveSparkPlan (17)
+- == Final Plan ==
   * BroadcastHashJoin Inner BuildLeft (12)
   :- BroadcastQueryStage (8)
   :  +- BroadcastExchange (7)
   :     +- * Filter (6)
   :        +- * ColumnarToRow (5)
   :           +- InMemoryTableScan (1)
   :                 +- InMemoryRelation (2)
   :                       +- * ColumnarToRow (4)
   :                          +- Scan parquet default.t1 (3)
   +- * Filter (11)
      +- * ColumnarToRow (10)
         +- Scan parquet default.t2 (9)
+- == Initial Plan ==
   BroadcastHashJoin Inner BuildLeft (16)
   :- BroadcastExchange (14)
   :  +- Filter (13)
   :     +- InMemoryTableScan (1)
   :           +- InMemoryRelation (2)
   :                 +- * ColumnarToRow (4)
   :                    +- Scan parquet default.t1 (3)
   +- Filter (15)
      +- Scan parquet default.t2 (9)

(1) InMemoryTableScan
Output [1]: [k#x]
Arguments: [k#x], [isnotnull(k#x)]

(2) InMemoryRelation
Arguments: [k#x], CachedRDDBuilder(org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer3ccb12d,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) ColumnarToRow
+- FileScan parquet default.t1[k#x] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/Users/mike.chen/code/apacheSpark/spark/spark-warehouse/org.apach..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<k:int>
,None)

(3) Scan parquet default.t1
Output [1]: [k#x]
Batched: true
Location: InMemoryFileIndex [file:/Users/mike.chen/code/apacheSpark/spark/spark-warehouse/org.apache.spark.sql.ExplainSuiteAE/t1]
ReadSchema: struct<k:int>

(4) ColumnarToRow [codegen id : 1]
Input [1]: [k#x]

(5) ColumnarToRow [codegen id : 1]
Input [1]: [k#x]

(6) Filter [codegen id : 1]
Input [1]: [k#x]
Condition : isnotnull(k#x)

(7) BroadcastExchange
Input [1]: [k#x]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#x]

(8) BroadcastQueryStage
Output [1]: [k#x]
Arguments: 0

(9) Scan parquet default.t2
Output [1]: [key#x]
Batched: true
Location: InMemoryFileIndex [file:/Users/mike.chen/code/apacheSpark/spark/spark-warehouse/org.apache.spark.sql.ExplainSuiteAE/t2]
PushedFilters: [IsNotNull(key)]
ReadSchema: struct<key:int>

(10) ColumnarToRow
Input [1]: [key#x]

(11) Filter
Input [1]: [key#x]
Condition : isnotnull(key#x)

(12) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [k#x]
Right keys [1]: [key#x]
Join condition: None

(13) Filter
Input [1]: [k#x]
Condition : isnotnull(k#x)

(14) BroadcastExchange
Input [1]: [k#x]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#x]

(15) Filter
Input [1]: [key#x]
Condition : isnotnull(key#x)

(16) BroadcastHashJoin
Left keys [1]: [k#x]
Right keys [1]: [key#x]
Join condition: None

(17) AdaptiveSparkPlan
Output [2]: [k#x, key#x]
Arguments: isFinalPlan=true
```

### How was this patch tested?

add test

Closes #34036 from ChenMichael/SPARK-36795-Duplicate-node-id-with-inMemoryRelation.

Authored-by: Michael Chen <mike.chen@workday.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-23 15:54:33 +09:00
Hyukjin Kwon 0076eba8d0 [MINOR][SQL][DOCS] Correct the 'options' description on UnresolvedRelation
### What changes were proposed in this pull request?

This PR fixes the 'options' description on `UnresolvedRelation`. This comment was added in https://github.com/apache/spark/pull/29535 but not valid anymore because V1 also uses this `options` (and merge the options with the table properties) per https://github.com/apache/spark/pull/29712.

This PR can go through from `master` to `branch-3.1`.

### Why are the changes needed?

To make `UnresolvedRelation.options`'s description clearer.

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

No, dev-only.

### How was this patch tested?

Scala linter by `dev/linter-scala`.

Closes #34075 from HyukjinKwon/minor-comment-unresolved-releation.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Huaxin Gao <huaxin_gao@apple.com>
2021-09-22 23:00:15 -07:00
Karen Feng 7cc9667c88 [SPARK-36405][SQL][TESTS] Check that SQLSTATEs are valid
### What changes were proposed in this pull request?

Adds validation that the SQLSTATEs in the error class JSON are a subset of those provided in the README.

### Why are the changes needed?

Validation of error class JSON

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

No

### How was this patch tested?

Unit test

Closes #33627 from karenfeng/check-sqlstates.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-23 14:24:59 +09:00
itholic 5268904742 [SPARK-36506][PYTHON] Improve test coverage for series.py and indexes/*.py
### What changes were proposed in this pull request?

This PR proposes improving test coverage for pandas-on-Spark Series & Index code base, which is written in `series.py` and `indexes/*.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 new warning for `__xor__`, `__and__`, `__or__`.

### 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 #33844 from itholic/SPARK-36506.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-23 14:23:52 +09:00
Fabian A.J. Thiele 4ea54e8672 [SPARK-36782][CORE] Avoid blocking dispatcher-BlockManagerMaster during UpdateBlockInfo
### What changes were proposed in this pull request?
Delegate potentially blocking call to `mapOutputTracker.updateMapOutput` from within  `UpdateBlockInfo` from `dispatcher-BlockManagerMaster` to the threadpool to avoid blocking the endpoint. This code path is only accessed for `ShuffleIndexBlockId`, other blocks are still executed on the `dispatcher-BlockManagerMaster` itself.

Change `updateBlockInfo` to return `Future[Boolean]` instead of `Boolean`. Response will be sent to RPC caller upon successful completion of the future.

Introduce a unit test that forces `MapOutputTracker` to make a broadcast as part of `MapOutputTracker.serializeOutputStatuses` when running decommission tests.

### Why are the changes needed?
[SPARK-36782](https://issues.apache.org/jira/browse/SPARK-36782) describes a deadlock occurring if the `dispatcher-BlockManagerMaster` is allowed to block while waiting for write access to data structures.

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

### How was this patch tested?
Unit test as introduced in this PR.

---

Ping eejbyfeldt for notice.

Closes #34043 from f-thiele/SPARK-36782.

Lead-authored-by: Fabian A.J. Thiele <fabian.thiele@posteo.de>
Co-authored-by: Emil Ejbyfeldt <eejbyfeldt@liveintent.com>
Co-authored-by: Fabian A.J. Thiele <fthiele@liveintent.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-09-23 12:56:49 +08:00
allisonwang-db 4a8dc5f7a3 [SPARK-36747][SQL] Do not collapse Project with Aggregate when correlated subqueries are present in the project list
### What changes were proposed in this pull request?
This PR adds a check in the optimizer rule `CollapseProject` to avoid combining Project with Aggregate when the project list contains one or more correlated scalar subqueries that reference the output of the aggregate. Combining Project with Aggregate can lead to an invalid plan after correlated subquery rewrite. This is because correlated scalar subqueries' references are used as join conditions, which cannot host aggregate expressions.

For example
```sql
select (select sum(c2) from t where c1 = cast(s as int)) from (select sum(c2) s from t)
```

```
== Optimized Logical Plan ==
Aggregate [sum(c2)#10L AS scalarsubquery(s)#11L] <--- Aggregate has neither grouping nor aggregate expressions.
+- Project [sum(c2)#10L]
   +- Join LeftOuter, (c1#2 = cast(sum(c2#3) as int))  <--- Aggregate expression in join condition
      :- LocalRelation [c2#3]
      +- Aggregate [c1#2], [sum(c2#3) AS sum(c2)#10L, c1#2]
         +- LocalRelation [c1#2, c2#3]

java.lang.UnsupportedOperationException: Cannot generate code for expression: sum(input[0, int, false])
```
Currently, we only allow a correlated scalar subquery in Aggregate if it is also in the grouping expressions.
079a9c5292/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala (L661-L666)

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

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

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

Closes #33990 from allisonwang-db/spark-36747-collapse-agg.

Authored-by: allisonwang-db <allison.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-23 12:50:27 +08:00
jiaoqb 8a1a91bd71 [SPARK-36791][DOCS] Fix spelling mistakes in running-on-yarn.md file where JHS_POST should be JHS_HOST
### What changes were proposed in this pull request?
The PR fixes SPARK-36791 by replacing JHS_POST with JHS_HOST

### Why are the changes needed?
There are spelling mistakes in running-on-yarn.md file where JHS_POST should be JHS_HOST

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

### How was this patch tested?
Not needed for docs

Closes #34031 from jiaoqingbo/jiaoqingbo.

Authored-by: jiaoqb <jiaoqb@asiainfo.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-23 12:47:38 +09:00
Yuto Akutsu 04348574ab [SPARK-36824][R] Add sec and csc as R functions
### What changes were proposed in this pull request?

Add secant and cosecant as R functions.

### Why are the changes needed?

[SEC and CSC have been added](https://github.com/apache/spark/pull/33988) thus these functions need R support.

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

Yes, users can now call those functions as R functions.

### How was this patch tested?

unit tests added.

Closes #34067 from yutoacts/SPARK-36824.

Authored-by: Yuto Akutsu <yuto.akutsu@oss.nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-23 10:59:40 +09:00
Hyukjin Kwon cc2fcb4794 [SPARK-36708][PYTHON] Support numpy.typing for annotating ArrayType in pandas API on Spark
### What changes were proposed in this pull request?

This PR adds the support of understanding `numpy.typing` package that's added from NumPy 1.21.

### Why are the changes needed?

For user-friendly return type specification in type hints for function apply APIs in pandas API on Spark.

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

Yes, this PR will enable users to specify return type as `numpy.typing.NDArray[...]` to internally specify pandas UDF's return type.

For example,

```python
import pandas as pd
import pyspark.pandas as ps

pdf = pd.DataFrame(
    {"a": [1, 2, 3, 4, 5, 6, 7, 8, 9], "b": [[e] for e in [4, 5, 6, 3, 2, 1, 0, 0, 0]]},
    index=np.random.rand(9),
)
psdf = ps.from_pandas(pdf)

def func(x) -> ps.DataFrame[float, [int, ntp.NDArray[int]]]:
    return x

psdf.pandas_on_spark.apply_batch(func)
```

### How was this patch tested?

Unittest and e2e tests were added.

Closes #34028 from HyukjinKwon/SPARK-36708.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-23 10:50:10 +09:00
Xinrong Meng 6a5ee0283c [SPARK-36818][PYTHON] Fix filtering a Series by a boolean Series
### What changes were proposed in this pull request?
Fix filtering a Series (without a name) by a boolean Series.

### Why are the changes needed?
A bugfix. The issue is raised as https://github.com/databricks/koalas/issues/2199.

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

#### From
```py
>>> psser = ps.Series([0, 1, 2, 3, 4])
>>> ps.set_option('compute.ops_on_diff_frames', True)
>>> psser.loc[ps.Series([True, True, True, False, False])]
Traceback (most recent call last):
...
KeyError: 'none key'

```

#### To
```py
>>> psser = ps.Series([0, 1, 2, 3, 4])
>>> ps.set_option('compute.ops_on_diff_frames', True)
>>> psser.loc[ps.Series([True, True, True, False, False])]
0    0
1    1
2    2
dtype: int64
```

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

Closes #34061 from xinrong-databricks/filter_series.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-09-22 12:52:56 -07:00
Angerszhuuuu a7cbe69986 [SPARK-36753][SQL] ArrayExcept handle duplicated Double.NaN and Float.NaN
### What changes were proposed in this pull request?
For query
```
select array_except(array(cast('nan' as double), 1d), array(cast('nan' as double)))
```
This returns [NaN, 1d], but it should return [1d].
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?
ArrayExcept won't show handle equal `NaN` value

### How was this patch tested?
Added UT

Closes #33994 from AngersZhuuuu/SPARK-36753.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-22 23:51:41 +08:00
Ivan Sadikov ec26d94eac [SPARK-36803][SQL] Fix ArrayType conversion when reading Parquet files written in legacy mode
### What changes were proposed in this pull request?

This PR fixes an issue when reading of a Parquet file written with legacy mode would fail due to incorrect Parquet LIST to ArrayType conversion.

The issue arises when using schema evolution and utilising the parquet-mr reader. 2-level LIST annotated types could be parsed incorrectly as 3-level LIST annotated types because their underlying element type does not match the full inferred Catalyst schema.

### Why are the changes needed?

It appears to be a long-standing issue with the legacy mode due to the imprecise check in ParquetRowConverter that was trying to determine Parquet backward compatibility using Catalyst schema: `DataType.equalsIgnoreCompatibleNullability(guessedElementType, elementType)` in https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala#L606.

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

### How was this patch tested?

Added a new test case in ParquetInteroperabilitySuite.scala.

Closes #34044 from sadikovi/parquet-legacy-write-mode-list-issue.

Authored-by: Ivan Sadikov <ivan.sadikov@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-22 17:40:39 +08:00
ulysses-you d90b479185 [SPARK-36822][SQL] BroadcastNestedLoopJoinExec should use all condition instead of non-equi condition
### What changes were proposed in this pull request?

Change `nonEquiCond` to all join condition at `JoinSelection.ExtractEquiJoinKeys` pattern.

### Why are the changes needed?

At `JoinSelection`, with `ExtractEquiJoinKeys`, we use `nonEquiCond` as the join condition. It's wrong since there should exist some equi condition.
```
Seq(joins.BroadcastNestedLoopJoinExec(
  planLater(left), planLater(right), buildSide, joinType, nonEquiCond))
```
But it's should not be a bug, since we always use the smj as the default join strategy for ExtractEquiJoinKeys.

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

no

### How was this patch tested?

it's not a bug for now, but we should fix it in case we use this code path in future.

Closes #34065 from ulysses-you/join-condition.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-22 17:25:56 +08:00
Huaxin Gao 1d7d972021 [SPARK-36760][SQL] Add interface SupportsPushDownV2Filters
Co-Authored-By: DB Tsai d_tsaiapple.com
Co-Authored-By: Huaxin Gao huaxin_gaoapple.com
### What changes were proposed in this pull request?
This is the 2nd PR for V2 Filter support. This PR does the following:

- Add interface SupportsPushDownV2Filters

Future work:
- refactor `OrcFilters`, `ParquetFilters`, `JacksonParser`, `UnivocityParser` so both V1 file source and V2 file source can use them
- For V2 file source: implement  v2 filter -> parquet/orc filter. csv and Json don't have real filters, but also need to change the current code to have v2 filter -> `JacksonParser`/`UnivocityParser`
- For V1 file source, keep what we currently have: v1 filter -> parquet/orc filter
- We don't need v1filter.toV2 and v2filter.toV1 since we have two separate paths

The reasons that we have reached the above conclusion:
- The major motivation to implement V2Filter is to eliminate the unnecessary conversion between Catalyst types and Scala types when using Filters.
- We provide this `SupportsPushDownV2Filters` in this PR so V2 data source (e.g. iceberg) can implement it and use V2 Filters
- There are lots of work to implement v2 filters in the V2 file sources because of the following reasons:

possible approaches for implementing V2Filter:
1. keep what we have for file source v1: v1 filter -> parquet/orc filter
    file source v2 we will implement v2 filter -> parquet/orc filter
    We don't need v1->v2 and v2->v1
    problem with this approach: there are lots of code duplication

2.  We will implement v2 filter -> parquet/orc filter
     file source v1: v1 filter -> v2 filter -> parquet/orc filter
     We will need V1 -> V2
     This is the approach I am using in https://github.com/apache/spark/pull/33973
     In that PR, I have
     v2 orc: v2 filter -> orc filter
     V1 orc: v1 -> v2 -> orc filter

     v2 csv: v2->v1, new UnivocityParser
     v1 csv: new UnivocityParser

    v2 Json: v2->v1, new JacksonParser
    v1 Json: new JacksonParser

    csv and Json don't have real filters, they just use filter references, should be OK to use either v1 and v2. Easier to use
    v1 because no need to change.

    I haven't finished parquet yet. The PR doesn't have the parquet V2Filter implementation, but I plan to have
    v2 parquet: v2 filter -> parquet filter
    v1 parquet: v1 -> v2 -> parquet filter

    Problem with this approach:
    1. It's not easy to implement V1->V2  because V2 filter have `LiteralValue` and needs type info. We already lost the type information when we convert Expression filer to v1 filter.
    2. parquet is OK
        Use Timestamp as example, parquet filter takes long for timestamp
        v2 parquet: v2 filter -> parquet filter
       timestamp
       Expression (Long) -> v2 filter (LiteralValue  Long)-> parquet filter (Long)

       V1 parquet: v1 -> v2 -> parquet filter
       timestamp
       Expression (Long) -> v1 filter (timestamp) -> v2 filter (LiteralValue  Long)-> parquet filter (Long)

       but we have problem for orc because orc filter takes java Timestamp
       v2 orc: v2 filter -> orc filter
       timestamp
       Expression (Long) -> v2 filter (LiteralValue  Long)->  parquet filter (Timestamp)

       V1 orc: v1 -> v2 -> orc filter
       Expression (Long) ->  v1 filter (timestamp) -> v2 filter (LiteralValue  Long)-> parquet filter (Timestamp)
      This defeats the purpose of implementing v2 filters.
3.  keep what we have for file source v1: v1 filter -> parquet/orc filter
     file source v2: v2 filter -> v1 filter -> parquet/orc filter
     We will need V2 -> V1
     we have similar problem as approach 2.

So the conclusion is: approach 1 (keep what we have for file source v1: v1 filter -> parquet/orc filter
    file source v2 we will implement v2 filter -> parquet/orc filter) is better, but there are lots of code duplication. We will need to refactor `OrcFilters`, `ParquetFilters`, `JacksonParser`, `UnivocityParser` so both V1 file source and V2 file source can use them.

### Why are the changes needed?
Use V2Filters to eliminate the unnecessary conversion between Catalyst types and Scala types.

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

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

Closes #34001 from huaxingao/v2filter.

Lead-authored-by: Huaxin Gao <huaxin_gao@apple.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-22 16:58:13 +08:00
Chao Sun 6eb7559901 [SPARK-36820][SQL] Disable tests related to LZ4 for Hadoop 2.7 profile
### What changes were proposed in this pull request?

Disable tests related to LZ4 in `FileSourceCodecSuite`, `FileSuite` and `ParquetCompressionCodecPrecedenceSuite` when using `hadoop-2.7` profile.
### Why are the changes needed?

At the moment, parquet-mr uses LZ4 compression codec provided by Hadoop, and only since HADOOP-17292 (in 3.3.1/3.4.0) the latter added `lz4-java` to remove the restriction that the codec can only be run with native library. As consequence, the test will fail when using `hadoop-2.7` profile.

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

No, it's just test.

### How was this patch tested?

Existing test

Closes #34064 from sunchao/SPARK-36820.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-22 00:12:29 -07:00
Xinrong Meng 079a9c5292 [SPARK-36771][PYTHON] Fix pop of Categorical Series
### What changes were proposed in this pull request?
Fix `pop` of Categorical Series to be consistent with the latest pandas (1.3.2) behavior.

### Why are the changes needed?
As https://github.com/databricks/koalas/issues/2198, pandas API on Spark behaves differently from pandas on `pop` of Categorical Series.

### Does this PR introduce _any_ user-facing change?
Yes, results of `pop` of Categorical Series change.

#### From
```py
>>> psser = ps.Series(["a", "b", "c", "a"], dtype="category")
>>> psser
0    a
1    b
2    c
3    a
dtype: category
Categories (3, object): ['a', 'b', 'c']
>>> psser.pop(0)
0
>>> psser
1    b
2    c
3    a
dtype: category
Categories (3, object): ['a', 'b', 'c']
>>> psser.pop(3)
0
>>> psser
1    b
2    c
dtype: category
Categories (3, object): ['a', 'b', 'c']
```

#### To
```py
>>> psser = ps.Series(["a", "b", "c", "a"], dtype="category")
>>> psser
0    a
1    b
2    c
3    a
dtype: category
Categories (3, object): ['a', 'b', 'c']
>>> psser.pop(0)
'a'
>>> psser
1    b
2    c
3    a
dtype: category
Categories (3, object): ['a', 'b', 'c']
>>> psser.pop(3)
'a'
>>> psser
1    b
2    c
dtype: category
Categories (3, object): ['a', 'b', 'c']

```

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

Closes #34052 from xinrong-databricks/cat_pop.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-09-21 14:11:21 -07:00
Angerszhuuuu b7d99e3eea [SPARK-36615][CORE] Register shutdown hook earlier when start SC
### What changes were proposed in this pull request?
Since user always use ctrl+c to stop a starting SC when register with yarn in client mode when resources are tight.

In this time, SC have not register the Shutdown hook, this cause we won't invoke `sc.stop()` when exit the application.
We should register the ShutdownHook earlier when starting a SparkContext.

### Why are the changes needed?

Make sure we will invoke `sc.stop()` when kill a starting SparkContext application.

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

### How was this patch tested?
Not need

Closes #33869 from AngersZhuuuu/SPARK-36615.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-09-21 13:23:14 -05:00
Gengliang Wang ba5708d944 [SPARK-36670][FOLLOWUP][TEST] Remove brotli-codec dependency
### What changes were proposed in this pull request?

Remove `com.github.rdblue:brotli-codec:0.1.1` dependency.

### Why are the changes needed?

As Stephen Coy pointed out in the dev list, we should not have `com.github.rdblue:brotli-codec:0.1.1` dependency which is not available on Maven Central. This is to avoid possible artifact changes on `Jitpack.io`.
Also, the dependency is for tests only. I suggest that we remove it now to unblock the 3.2.0 release ASAP.

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

No

### How was this patch tested?

GA tests.

Closes #34059 from gengliangwang/removeDeps.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-21 10:57:20 -07:00
Xinrong Meng 33e463ccf9 [SPARK-36769][PYTHON] Improve filter of single-indexed DataFrame
### What changes were proposed in this pull request?
Improve `filter` of single-indexed DataFrame by replacing a long Project with Filter or Join.

### Why are the changes needed?
When the given `items` have too many elements, a long Project is introduced.
We may replace that with `Column.isin` or joining depending on the length of `items` for better performance.

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

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

Closes #33998 from xinrong-databricks/impr_filter.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-09-21 10:20:15 -07:00
Yufei Gu 688b95b136 [SPARK-36814][SQL] Make class ColumnarBatch extendable
### What changes were proposed in this pull request?
Change class ColumnarBatch to a non-final class

### Why are the changes needed?
To support better vectorized reading in multiple data source, ColumnarBatch need to be extendable. For example, To support row-level delete(  https://github.com/apache/iceberg/issues/3141) in Iceberg's vectorized read, we need to filter out deleted rows in a batch, which requires ColumnarBatch to be extendable.

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

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

Closes #34054 from flyrain/columnarbatch-extendable.

Authored-by: Yufei Gu <yufei_gu@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2021-09-21 15:24:55 +00:00
Max Gekk d2340f8e1c [SPARK-36807][SQL] Merge ANSI interval types to a tightest common type
### What changes were proposed in this pull request?
In the PR, I propose to modify `StructType` to support merging of ANSI interval types with different fields.

### Why are the changes needed?
This will allow merging of schemas from different datasource files.

### Does this PR introduce _any_ user-facing change?
No, the ANSI interval types haven't released yet.

### How was this patch tested?
Added new test to `StructTypeSuite`.

Closes #34049 from MaxGekk/merge-ansi-interval-types.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-09-21 10:20:16 +03:00
dgd-contributor cc182fe6f6 [SPARK-36785][PYTHON] Fix DataFrame.isin when DataFrame has NaN value
### What changes were proposed in this pull request?
Fix DataFrame.isin when DataFrame has NaN value

### Why are the changes needed?
Fix DataFrame.isin when DataFrame has NaN value

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

>>> psdf.isin(other)
      a     b     c
0  None  None  True
1  True  None  None
2  None  None  True
3  None  None  None
4  None  True  True
5  None  True  True
6  None  None  True
7  None  None  None
8  None  None  None

>>> psdf.to_pandas().isin(other)
       a      b      c
0  False  False   True
1   True  False  False
2  False  False   True
3  False  False  False
4  False   True   True
5  False   True   True
6  False  False   True
7  False  False  False
8  False  False  False
```

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

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

>>> psdf.isin(other)
       a      b      c
0  False  False   True
1   True  False  False
2  False  False   True
3  False  False  False
4  False   True   True
5  False   True   True
6  False  False   True
7  False  False  False
8  False  False  False
```

### How was this patch tested?
Unit tests

Closes #34040 from dgd-contributor/SPARK-36785_dataframe.isin_fix.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-09-20 17:52:51 -07:00
Xinrong Meng 4b61c623b5 [SPARK-36746][PYTHON] Refactor _select_rows_by_iterable in iLocIndexer to use Column.isin
### What changes were proposed in this pull request?
Refactor `_select_rows_by_iterable` in `iLocIndexer` to use `Column.isin`.

### Why are the changes needed?
For better performance.

After a rough benchmark, a long projection performs worse than `Column.isin`, even when the length of the filtering conditions exceeding `compute.isin_limit`.

So we use `Column.isin` instead.

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

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

Closes #33964 from xinrong-databricks/iloc_select.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-09-20 15:00:10 -07:00
Xinrong Meng 4cf86d33ad [SPARK-36618][PYTHON] Support dropping rows of a single-indexed DataFrame
### What changes were proposed in this pull request?
Support dropping rows of a single-indexed DataFrame.

Dropping rows and columns at the same time is supported in this PR  as well.

### Why are the changes needed?
To increase pandas API coverage.

### Does this PR introduce _any_ user-facing change?
Yes, dropping rows of a single-indexed DataFrame is supported now.

```py
>>> df = ps.DataFrame(np.arange(12).reshape(3, 4), columns=['A', 'B', 'C', 'D'])
>>> df
   A  B   C   D
0  0  1   2   3
1  4  5   6   7
2  8  9  10  11
```
#### From
```py
>>> df.drop([0, 1])
Traceback (most recent call last):
...
KeyError: [(0,), (1,)]

>>> df.drop([0, 1], axis=0)
Traceback (most recent call last):
...
NotImplementedError: Drop currently only works for axis=1

>>> df.drop(1)
Traceback (most recent call last):
...
KeyError: [(1,)]

>>> df.drop(index=1)
Traceback (most recent call last):
...
TypeError: drop() got an unexpected keyword argument 'index'

>>> df.drop(index=[0, 1], columns='A')
Traceback (most recent call last):
...
TypeError: drop() got an unexpected keyword argument 'index'
```
#### To
```py
>>> df.drop([0, 1])
   A  B   C   D
2  8  9  10  11

>>> df.drop([0, 1], axis=0)
   A  B   C   D
2  8  9  10  11

>>> df.drop(1)
   A  B   C   D
0  0  1   2   3
2  8  9  10  11

>>> df.drop(index=1)
   A  B   C   D
0  0  1   2   3
2  8  9  10  11

>>> df.drop(index=[0, 1], columns='A')
   B   C   D
2  9  10  11
```

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

Closes #33929 from xinrong-databricks/frame_drop.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-09-20 14:50:50 -07:00
Dongjoon Hyun a178752540 [SPARK-36806][K8S][R] Use R 4.0.4 in K8s R image
### What changes were proposed in this pull request?

This PR aims to upgrade R from 3.6.3 to 4.0.4 in K8s R Docker image.

### Why are the changes needed?

`openjdk:11-jre-slim` image is upgraded to `Debian 11`.

```
$ docker run -it openjdk:11-jre-slim cat /etc/os-release
PRETTY_NAME="Debian GNU/Linux 11 (bullseye)"
NAME="Debian GNU/Linux"
VERSION_ID="11"
VERSION="11 (bullseye)"
VERSION_CODENAME=bullseye
ID=debian
HOME_URL="https://www.debian.org/"
SUPPORT_URL="https://www.debian.org/support"
BUG_REPORT_URL="https://bugs.debian.org/"
```

It causes `R 3.5` installation failures in our K8s integration test environment.
- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/47953/
```
The following packages have unmet dependencies:
 r-base-core : Depends: libicu63 (>= 63.1-1~) but it is not installable
               Depends: libreadline7 (>= 6.0) but it is not installable
E: Unable to correct problems, you have held broken packages.
The command '/bin/sh -c apt-get update &&   apt install -y gnupg &&   echo "deb http://cloud.r-project.org/bin/linux/debian buster-cran35/" >> /etc/apt/sources.list &&   apt-key adv --keyserver keyserver.ubuntu.com --recv-key 'E19F5F87128899B192B1A2C2AD5F960A256A04AF' &&   apt-get update &&
apt install -y -t buster-cran35 r-base r-base-dev &&   rm -rf
```

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

Yes, this will recover the installation.

### How was this patch tested?

Succeed to build SparkR docker image in the K8s integration test in Jenkins CI.

- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/47959/
```
Successfully built 32e1a0cd5ff8
Successfully tagged kubespark/spark-r:3.3.0-SNAPSHOT_6e4f7e2d-054d-4978-812f-4f32fc546b51
```

Closes #34048 from dongjoon-hyun/SPARK-36806.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-20 10:52:45 -07:00
Kousuke Saruta 75e71ef7a8 [SPARK-36808][BUILD] Upgrade Kafka to 2.8.1
### What changes were proposed in this pull request?

This PR upgrades Kafka from `2.8.0` to `2.8.1`.

### Why are the changes needed?

Kafka `2.8.1` was released a few hours ago, which includes a bunch of bug fix.
https://downloads.apache.org/kafka/2.8.1/RELEASE_NOTES.html

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

No.

### How was this patch tested?

CIs.

Closes #34050 from sarutak/upgrade-kafka-2.8.1.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-20 10:13:43 -07:00
Dongjoon Hyun 3b560390c0 [SPARK-36805][BUILD][K8S] Upgrade kubernetes-client to 5.7.3
### What changes were proposed in this pull request?

This PR aims to upgrade `kubernetes-client` from 5.6.0 to 5.7.3

### Why are the changes needed?

This will bring the latest improvements and bug fixes.
- https://github.com/fabric8io/kubernetes-client/releases/tag/v5.7.3
- https://github.com/fabric8io/kubernetes-client/releases/tag/v5.7.2
- https://github.com/fabric8io/kubernetes-client/releases/tag/v5.7.1
- https://github.com/fabric8io/kubernetes-client/releases/tag/v5.7.0

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

No

### How was this patch tested?

Pass the CIs.

Closes #34047 from dongjoon-hyun/SPARK-36805.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-20 09:14:48 -07:00
Yuto Akutsu 30d17b6333 [SPARK-36683][SQL] Add new built-in SQL functions: SEC and CSC
### What changes were proposed in this pull request?

Add new built-in SQL functions: secant and cosecant, and add them as Scala and Python functions.

### Why are the changes needed?

Cotangent has been supported in Spark SQL but Secant and Cosecant are missing though I believe they can be used as much as cot.
Related Links: [SPARK-20751](https://github.com/apache/spark/pull/17999) [SPARK-36660](https://github.com/apache/spark/pull/33906)

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

Yes, users can now use these functions.

### How was this patch tested?

Unit tests

Closes #33988 from yutoacts/SPARK-36683.

Authored-by: Yuto Akutsu <yuto.akutsu@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-09-20 22:38:47 +09:00
dgd-contributor 4cc39cfe15 [SPARK-36101][CORE] Grouping exception in core/api
### What changes were proposed in this pull request?
This PR group exception messages in core/src/main/scala/org/apache/spark/api

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #33536 from dgd-contributor/SPARK-36101.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-20 17:19:29 +08:00
Angerszhuuuu 2fc7f2f702 [SPARK-36754][SQL] ArrayIntersect handle duplicated Double.NaN and Float.NaN
### What changes were proposed in this pull request?
For query
```
select array_intersect(array(cast('nan' as double), 1d), array(cast('nan' as double)))
```
This returns [NaN], but it should return [].
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?
ArrayIntersect won't show equal `NaN` value

### How was this patch tested?
Added UT

Closes #33995 from AngersZhuuuu/SPARK-36754.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-20 16:48:59 +08:00
Dongjoon Hyun a396dd6216 [SPARK-34112][BUILD] Upgrade ORC to 1.7.0
### What changes were proposed in this pull request?

This PR aims to upgrade Apache ORC from 1.6.11 to 1.7.0 for Apache Spark 3.3.0.

### Why are the changes needed?

[Apache ORC 1.7.0](https://orc.apache.org/news/2021/09/15/ORC-1.7.0/) is a new release with the following new features and improvements.
  - ORC-377 Support Snappy compression in C++ Writer
  - ORC-577 Support row-level filtering
  - ORC-716 Build and test on Java 17-EA
  - ORC-731 Improve Java Tools
  - ORC-742 LazyIO of non-filter columns
  - ORC-751 Implement Predicate Pushdown in C++ Reader
  - ORC-755 Introduce OrcFilterContext
  - ORC-757 Add Hashtable implementation for dictionary
  - ORC-780 Support LZ4 Compression in C++ Writer
  - ORC-797 Allow writers to get the stripe information
  - ORC-818 Build and test in Apple Silicon
  - ORC-861 Bump CMake minimum requirement to 2.8.12
  - ORC-867 Upgrade hive-storage-api to 2.8.1
  - ORC-984 Save the software version that wrote each ORC file

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

No.

### How was this patch tested?

Pass the existing CIs because this is a dependency change.

Closes #34045 from dongjoon-hyun/SPARK-34112.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-20 01:09:15 -07:00
Hyukjin Kwon 8d8b4aabc3 [SPARK-36710][PYTHON] Support new typing syntax in function apply APIs in pandas API on Spark
### What changes were proposed in this pull request?

This PR proposes the new syntax introduced in https://github.com/apache/spark/pull/33954. Namely, users now can specify the index type and name as below:

```python
import pandas as pd
import pyspark.pandas as ps
def transform(pdf) -> pd.DataFrame[int, [int, int]]:
    pdf['A'] = pdf.id + 1
    return pdf

ps.range(5).koalas.apply_batch(transform)
```

```
   c0  c1
0   0   1
1   1   2
2   2   3
3   3   4
4   4   5
```

```python
import pandas as pd
import pyspark.pandas as ps
def transform(pdf) -> pd.DataFrame[("index", int), [("a", int), ("b", int)]]:
    pdf['A'] = pdf.id * pdf.id
    return pdf

ps.range(5).koalas.apply_batch(transform)
```

```
       a   b
index
0      0   0
1      1   1
2      2   4
3      3   9
4      4  16
```

Again, this syntax remains experimental and this is a non-standard way apart from Python standard. We should migrate to proper typing once pandas supports it like `numpy.typing`.

### Why are the changes needed?

The rationale is described in https://github.com/apache/spark/pull/33954. In order to avoid unnecessary computation for default index or schema inference.

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

Yes, this PR affects the following APIs:

- `DataFrame.apply(..., axis=1)`
- `DataFrame.groupby.apply(...)`
- `DataFrame.pandas_on_spark.transform_batch(...)`
- `DataFrame.pandas_on_spark.apply_batch(...)`

Now they can specify the index type with the new syntax below:

```
DataFrame[index_type, [type, ...]]
DataFrame[(index_name, index_type), [(name, type), ...]]
DataFrame[dtype instance, dtypes instance]
DataFrame[(index_name, index_type), zip(names, types)]
```

### How was this patch tested?

Manually tested, and unittests were added.

Closes #34007 from HyukjinKwon/SPARK-36710.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-20 10:37:06 +09:00
PengLei c2881c5ee2 [SPARK-36107][SQL] Refactor first set of 20 query execution errors to use error classes
### What changes were proposed in this pull request?
Refactor some exceptions in QueryExecutionErrors to use error classes. as follows:
```
columnChangeUnsupportedError
logicalHintOperatorNotRemovedDuringAnalysisError
cannotEvaluateExpressionError
cannotGenerateCodeForExpressionError
cannotTerminateGeneratorError
castingCauseOverflowError
cannotChangeDecimalPrecisionError
invalidInputSyntaxForNumericError
cannotCastFromNullTypeError
cannotCastError
cannotParseDecimalError
simpleStringWithNodeIdUnsupportedError
evaluateUnevaluableAggregateUnsupportedError
dataTypeUnsupportedError
dataTypeUnsupportedError
failedExecuteUserDefinedFunctionError
divideByZeroError
invalidArrayIndexError
mapKeyNotExistError
rowFromCSVParserNotExpectedError
```

### Why are the changes needed?
[SPARK-36107](https://issues.apache.org/jira/browse/SPARK-36107)

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

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

Closes #33538 from Peng-Lei/SPARK-36017.

Lead-authored-by: PengLei <peng.8lei@gmail.com>
Co-authored-by: Lei Peng <peng.8lei@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-20 10:34:19 +09:00
Ye Zhou cabc36b54d [SPARK-36772] FinalizeShuffleMerge fails with an exception due to attempt id not matching
### What changes were proposed in this pull request?
Remove the appAttemptId from TransportConf, and parsing through SparkEnv.

### Why are the changes needed?
Push based shuffle will fail if there are any attemptId set in the SparkConf, as the attemptId is not set correctly in Driver.

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

### How was this patch tested?
Tested within our Yarn cluster. Without this PR, the Driver will fail to finalize the shuffle merge on all the mergers. After the patch, Driver can successfully finalize the shuffle merge and the push based shuffle can work fine.
Also with unit test to verify the attemptId is being set in the BlockStoreClient in Driver.

Closes #34018 from zhouyejoe/SPARK-36772.

Authored-by: Ye Zhou <yezhou@linkedin.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-09-18 15:51:57 +08:00
dgd-contributor 32b8512912 [SPARK-36762][PYTHON] Fix Series.isin when Series has NaN values
### What changes were proposed in this pull request?
Fix Series.isin when Series has NaN values

### Why are the changes needed?
Fix Series.isin when Series has NaN values
``` python
>>> pser = pd.Series([None, 5, None, 3, 2, 1, None, 0, 0])
>>> psser = ps.from_pandas(pser)
>>> pser.isin([1, 3, 5, None])
0    False
1     True
2    False
3     True
4    False
5     True
6    False
7    False
8    False
dtype: bool
>>> psser.isin([1, 3, 5, None])
0    None
1    True
2    None
3    True
4    None
5    True
6    None
7    None
8    None
dtype: object
```

### Does this PR introduce _any_ user-facing change?
After this PR
``` python
>>> pser = pd.Series([None, 5, None, 3, 2, 1, None, 0, 0])
>>> psser = ps.from_pandas(pser)
>>> psser.isin([1, 3, 5, None])
0    False
1     True
2    False
3     True
4    False
5     True
6    False
7    False
8    False
dtype: bool

```

### How was this patch tested?
unit tests

Closes #34005 from dgd-contributor/SPARK-36762_fix_series.isin_when_values_have_NaN.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-09-17 17:48:15 -07:00
Liang-Chi Hsieh f9644cc253 [SPARK-36673][SQL][FOLLOWUP] Remove duplicate test in DataFrameSetOperationsSuite
### What changes were proposed in this pull request?

As a followup of #34025 to remove duplicate test.

### Why are the changes needed?

To remove duplicate test.

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

No

### How was this patch tested?

Existing test.

Closes #34032 from viirya/remove.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-17 11:52:19 -07:00
Yang He 5d0889bf36 [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>
2021-09-17 08:54:49 -07:00
Kousuke Saruta cd1b7e1777 [SPARK-36663][SQL] Support number-only column names in ORC data sources
### What changes were proposed in this pull request?

This PR aims to support number-only column names in ORC data sources.
In the current master, with ORC datasource, we can write a DataFrame which contains such columns into ORC files.
```
spark.sql("SELECT 'a' as `1`").write.orc("/path/to/dest")
```

But reading the ORC files will fail.
```
spark.read.orc("/path/to/dest")
...
== SQL ==
struct<1:string>
-------^^^

  at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:265)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:126)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parseDataType(ParseDriver.scala:40)
  at org.apache.spark.sql.execution.datasources.orc.OrcUtils$.org$apache$spark$sql$execution$datasources$orc$OrcUtils$$toCatalystSchema(OrcUtils.scala:91)
```
The cause of this is `OrcUtils.toCatalystSchema` fails to handle such column names.
In `OrcUtils.toCatalystSchema`, `CatalystSqlParser` is used to create a instance of `StructType` which represents a schema but `CatalystSqlParser.parseDataType` fails to parse if a column name (and nested field) consists of only numbers.

### Why are the changes needed?

For better usability.

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

No.

### How was this patch tested?

New tests.

Closes #33915 from sarutak/fix-orc-schema-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-17 21:54:36 +08:00
Angerszhuuuu 69e006dd53 [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>
2021-09-17 21:42:08 +08:00