Commit graph

30551 commits

Author SHA1 Message Date
Takuya UESHIN 68b54b702c [SPARK-35473][PYTHON] Fix disallow_untyped_defs mypy checks for pyspark.pandas.groupby
### What changes were proposed in this pull request?

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

### Why are the changes needed?

We should enable more disallow_untyped_defs mypy checks.

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

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

### How was this patch tested?

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

Closes #33032 from ueshin/issues/SPARK-35473/disallow_untyped_defs_groupby.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-23 09:51:33 +09:00
Wenchen Fan 7a21e9c48f [SPARK-35858][SQL] SparkPlan.makeCopy should not set the active session
### What changes were proposed in this pull request?

We introduced `SparkSession.withActive` a while ago, and we use it when we need to run some code with a certain SparkSession as the active session.

Somehow we missed `SparkPlan.makeCopy`, which sets active session directly. This PR proposes to call `SparkSession.withActive` there.

### Why are the changes needed?

make sure we don't change the active session unexpectedly.

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

No. `makeCopy` is an internal function and I can't find a real case that this can change the active session. Mostly in an upper level, there is already a `SparkSession.withActive`, like `QueryExecution.executePhase`

### How was this patch tested?

existing tests

Closes #33029 from cloud-fan/minor1.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-23 09:50:59 +09:00
Wenchen Fan a2c1a55b1f [SPARK-35700][SQL][FOLLOWUP] Read schema from ORC files should strip CHAR/VARCHAR types
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/33001 , to provide a more direct fix.

The regression in 3.1 was caused by the fact that we changed the parser and allow the parser to return CHAR/VARCHAR type. We should have replaced CHAR/VARCHAR with STRING before the data type flows into the query engine, however, `OrcUtils` is missed.

When reading ORC files, at the task side we will read the real schema from ORC file metadata, then apply filter pushdown. For some reason, the implementation turns ORC schema to Spark schema before filter pushdown, and this step does not strip CHAR/VARCHAR. Note, for Parquet we use the Parquet schema directly in filter pushdown, and do not this have problem.

This PR proposes to replace the CHAR/VARCHAR with STRING when turning ORC schema to Spark schema.

### Why are the changes needed?

a more directly bug fix

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

no

### How was this patch tested?

existing tests

Closes #33030 from cloud-fan/help.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-22 13:50:49 -07:00
Takuya UESHIN c418803df7 [SPARK-35847][PYTHON] Manage InternalField in DataTypeOps.isnull
### What changes were proposed in this pull request?

Properly set `InternalField` for `DataTypeOps.isnull`.

### Why are the changes needed?

The result of `DataTypeOps.isnull` must always be non-nullable boolean.
We should manage `InternalField` for this case.

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

No.

### How was this patch tested?

Added some more tests.

Closes #33005 from ueshin/issues/SPARK-35847/isnull_field.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-06-22 12:54:01 -07:00
Li Zhang dfd7b026dc [SPARK-35800][SS] Improving GroupState testability by introducing TestGroupState
### What changes were proposed in this pull request?
Proposed changes in this pull request:

1. Introducing the `TestGroupState` interface which is inherited from `GroupState` so that testing related getters can be exposed in a controlled manner
2. Changing `GroupStateImpl` to inherit from `TestGroupState` interface, instead of directly from `GroupState`
3. Implementing `TestGroupState` object with `create()` method to forward inputs to the private `GroupStateImpl` constructor
4. User input validations have been added into `GroupStateImpl`'s `createForStreaming()` method to prevent users from creating invalid GroupState objects.
5. Replacing existing `GroupStateImpl` usages in sql pkg internal unit tests with the newly added `TestGroupState` to give user best practice about `TestGroupState` usage.

With the changes in this PR, the class hierarchy is changed from `GroupStateImpl` -> `GroupState` to `GroupStateImpl` -> `TestGroupState` -> `GroupState` (-> means inherits from)

### Why are the changes needed?
The internal `GroupStateImpl` implementation for the `GroupState` interface has no public constructors accessible outside of the sql pkg. However, the user-provided state transition function for `[map|flatMap]GroupsWithState` requires a `GroupState` object as the prevState input.

Currently, users are calling the Structured Streaming engine in their unit tests in order to instantiate such `GroupState` instances, which makes UTs cumbersome.

The proposed `TestGroupState` interface is to give users controlled access to the `GroupStateImpl` internal implementation to largely improve testability of Structured Streaming state transition functions.

**Usage Example**
```
import org.apache.spark.sql.streaming.TestGroupState

test(“Structured Streaming state update function”) {
  var prevState = TestGroupState.create[UserStatus](
    optionalState = Optional.empty[UserStatus],
    timeoutConf = EventTimeTimeout,
    batchProcessingTimeMs = 1L,
    eventTimeWatermarkMs = Optional.of(1L),
    hasTimedOut = false)

  val userId: String = ...
  val actions: Iterator[UserAction] = ...

  assert(!prevState.hasUpdated)

  updateState(userId, actions, prevState)

  assert(prevState.hasUpdated)
}
```

### Does this PR introduce _any_ user-facing change?
Yes, the `TestGroupState` interface and its corresponding `create()` factory function in its companion object are introduced in this pull request for users to use in unit tests.

### How was this patch tested?
- New unit tests are added
- Existing GroupState unit tests are updated

Closes #32938 from lizhangdatabricks/improve-group-state-testability.

Authored-by: Li Zhang <li.zhang@databricks.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2021-06-22 15:04:01 -04:00
Yikun Jiang 1c26433f1d [SPARK-35849][PYTHON] Make astype method data-type-based for DecimalOps
### What changes were proposed in this pull request?
Make DecimalOps astype data-type-based.

See more in:
https://github.com/apache/spark/pull/32821#issuecomment-861119905

### Why are the changes needed?
Make DecimalOps astype data-type-based.

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

### How was this patch tested?
Existing test NumOpsTest.test_astype in pyspark/pandas/tests/data_type_ops/test_num_ops.py

Closes #33009 from Yikun/SPARK-35849.

Authored-by: Yikun Jiang <yikunkero@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-06-22 10:41:22 -07:00
Hyukjin Kwon 27046582e4 [SPARK-35645][PYTHON][DOCS] Merge contents and remove obsolete pages in Getting Started section
### What changes were proposed in this pull request?

This PR revise the installation to describe `pip install pyspark[pandas_on_spark]` and removes pandas-on-Spark installation and videos/blogposts.

### Why are the changes needed?

pandas-on-Spark installation is merged to PySpark installation pages. For videos/blogposts, now this is named pandas API on Spark. Old Koalas blogposts and videos are obsolete.

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

To end users, no because the docs are not released yet.

### How was this patch tested?

I manually built the docs and checked the output

Closes #33018 from HyukjinKwon/SPARK-35645.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-22 09:36:27 -07:00
Gengliang Wang ce53b7199d [SPARK-35854][SQL] Improve the error message of to_timestamp_ntz with invalid format pattern
### What changes were proposed in this pull request?

When SQL function `to_timestamp_ntz` has invalid format pattern input, throw a runtime exception with hints for the valid patterns, instead of throwing an upgrade exception with suggestions to use legacy formatters.

### Why are the changes needed?

As discussed in https://github.com/apache/spark/pull/32995/files#r655148980, there is an error message saying
"You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'yyyy-MM-dd GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0"

This is not true for function to_timestamp_ntz, which only uses the Iso8601TimestampFormatter and added since Spark 3.2. We should improve it.

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

No, the new SQL function is not released yet.

### How was this patch tested?

Unit test

Closes #33019 from gengliangwang/improveError.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-22 23:45:54 +08:00
Lei Peng bc61b62a55 [SPARK-35727][SQL] Return INTERVAL DAY from dates subtraction
What changes were proposed in this pull request?

1. Change the return value type from DayTimeIntervalType(DAY, SECOND) to DayTimeIntervalType(DAY, DAY) of SubtractDates.

Why are the changes needed?
https://issues.apache.org/jira/browse/SPARK-35727

Does this PR introduce any user-facing change?
no

How was this patch tested?
existed ut test

Closes #32999 from Peng-Lei/SPARK-35727.

Lead-authored-by: Lei Peng <peng.8lei@gmail.com>
Co-authored-by: PengLei <18066542445@189.cn>
Co-authored-by: Peng-Lei <peng.8lei@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-22 13:43:25 +00:00
YangJie 6c05459600 [SPARK-35838][BUILD][TESTS] Ensure all modules can be maven test independently in Scala 2.13
### What changes were proposed in this pull request?
Similar to SPARK-35532, the main change of this pr is add `scala-2.13` profile to external/kafka-0-10-sql/pom.xml, external/avro/pom.xml and sql/hive-thriftserver/pom.xml,  the `scala-2.13` profile include dependency on `scala-parallel-collections_2.13`, then all(34) spark modules can maven test independently.

### Why are the changes needed?
Ensure alll(34) spark modules can be maven test independently in Scala 2.13

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

### How was this patch tested?
- Pass the GitHub Action Scala 2.13 job
- Manual test:

1. Execute
```
dev/change-scala-version.sh 2.13

mvn clean install -DskipTests -Phadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -Phive -Pscala-2.13
```

2. maven test `external/kafka-0-10-sql` module
```
mvn test -Phadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -Phive -Pscala-2.13 -pl external/kafka-0-10-sql
```

**before**

```
Discovery starting.
Discovery completed in 857 milliseconds.
Run starting. Expected test count is: 464
...
KafkaRelationSuiteV2:
- explicit earliest to latest offsets
- default starting and ending offsets
- explicit offsets
- default starting and ending offsets with headers
- timestamp provided for starting and ending
- timestamp provided for starting, offset provided for ending
- timestamp provided for ending, offset provided for starting
- timestamp provided for starting, ending not provided
- timestamp provided for ending, starting not provided
- global timestamp provided for starting and ending
- no matched offset for timestamp - startingOffsets
- preferences on offset related options
- no matched offset for timestamp - endingOffsets
*** RUN ABORTED ***
  java.lang.NoClassDefFoundError: scala/collection/parallel/TaskSupport
  at org.apache.spark.SparkContext.$anonfun$union$1(SparkContext.scala:1411)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
  at org.apache.spark.SparkContext.withScope(SparkContext.scala:788)
  at org.apache.spark.SparkContext.union(SparkContext.scala:1405)
  at org.apache.spark.sql.execution.UnionExec.doExecute(basicPhysicalOperators.scala:697)
  at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:182)
  at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:220)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:217)
  ...
  Cause: java.lang.ClassNotFoundException: scala.collection.parallel.TaskSupport
  at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
  at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
  at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352)
  at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
  at org.apache.spark.SparkContext.$anonfun$union$1(SparkContext.scala:1411)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
  at org.apache.spark.SparkContext.withScope(SparkContext.scala:788)
  at org.apache.spark.SparkContext.union(SparkContext.scala:1405)
  at org.apache.spark.sql.execution.UnionExec.doExecute(basicPhysicalOperators.scala:697)
  ...
```

**After**

```
Run completed in 33 minutes, 51 seconds.
Total number of tests run: 464
Suites: completed 31, aborted 0
Tests: succeeded 464, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

3. maven test `external/avro` module

```
mvn test -Phadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -Phive -Pscala-2.13 -pl external/avro
```

**before**

```
Discovery starting.
Discovery completed in 2 seconds, 765 milliseconds.
Run starting. Expected test count is: 255
AvroReadSchemaSuite:
- append column at the end
- hide column at the end
- append column into middle
- hide column in the middle
- add a nested column at the end of the leaf struct column
- add a nested column in the middle of the leaf struct column
- add a nested column at the end of the middle struct column
- add a nested column in the middle of the middle struct column
- hide a nested column at the end of the leaf struct column
- hide a nested column in the middle of the leaf struct column
- hide a nested column at the end of the middle struct column
- hide a nested column in the middle of the middle struct column
*** RUN ABORTED ***
  java.lang.NoClassDefFoundError: scala/collection/parallel/TaskSupport
  at org.apache.spark.SparkContext.$anonfun$union$1(SparkContext.scala:1411)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
  at org.apache.spark.SparkContext.withScope(SparkContext.scala:788)
  at org.apache.spark.SparkContext.union(SparkContext.scala:1405)
  at org.apache.spark.sql.execution.UnionExec.doExecute(basicPhysicalOperators.scala:697)
  at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:182)
  at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:220)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:217)
  ...
  Cause: java.lang.ClassNotFoundException: scala.collection.parallel.TaskSupport
  at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
  at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
  at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352)
  at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
  at org.apache.spark.SparkContext.$anonfun$union$1(SparkContext.scala:1411)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
  at org.apache.spark.SparkContext.withScope(SparkContext.scala:788)
  at org.apache.spark.SparkContext.union(SparkContext.scala:1405)
  at org.apache.spark.sql.execution.UnionExec.doExecute(basicPhysicalOperators.scala:697)
  ...
```

**After**

```
Run completed in 1 minute, 42 seconds.
Total number of tests run: 255
Suites: completed 12, aborted 0
Tests: succeeded 255, failed 0, canceled 0, ignored 2, pending 0
All tests passed.
```

4.  maven test `sql/hive-thriftserver` module

```
mvn test -Phadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -Phive -Pscala-2.13 -pl sql/hive-thriftserver
```

**before**

```
- union.sql *** FAILED ***
  "1  a
  1 a
  2 b
  2 b" did not contain "Exception" Exception did not match for query #2
  SELECT *
  FROM   (SELECT * FROM t1
          UNION ALL
          SELECT * FROM t1), expected: 1  a
  1 a
  2 b
  2 b, but got: java.sql.SQLException
  org.apache.hive.service.cli.HiveSQLException: Error running query: java.lang.NoClassDefFoundError: scala/collection/parallel/TaskSupport
    at org.apache.spark.sql.hive.thriftserver.HiveThriftServerErrors$.runningQueryError(HiveThriftServerErrors.scala:38)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:324)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3.$anonfun$run$2(SparkExecuteStatementOperation.scala:229)
    at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18)
    at org.apache.spark.sql.hive.thriftserver.SparkOperation.withLocalProperties(SparkOperation.scala:79)
    at org.apache.spark.sql.hive.thriftserver.SparkOperation.withLocalProperties$(SparkOperation.scala:63)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.withLocalProperties(SparkExecuteStatementOperation.scala:43)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3.run(SparkExecuteStatementOperation.scala:229)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3.run(SparkExecuteStatementOperation.scala:224)
    at java.security.AccessController.doPrivileged(Native Method)
    at javax.security.auth.Subject.doAs(Subject.java:422)
    at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1878)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2.run(SparkExecuteStatementOperation.scala:238)
    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
    at java.util.concurrent.FutureTask.run(FutureTask.java:266)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
    at java.lang.Thread.run(Thread.java:748)
  Caused by: java.lang.NoClassDefFoundError: scala/collection/parallel/TaskSupport
    at org.apache.spark.SparkContext.$anonfun$union$1(SparkContext.scala:1411)
    at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
    at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
    at org.apache.spark.SparkContext.withScope(SparkContext.scala:788)
    at org.apache.spark.SparkContext.union(SparkContext.scala:1405)
    at org.apache.spark.sql.execution.UnionExec.doExecute(basicPhysicalOperators.scala:697)
    at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:182)
    at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:220)
    at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
    at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:217)
    at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:178)
    at org.apache.spark.sql.execution.SparkPlan.getByteArrayRdd(SparkPlan.scala:323)
    at org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:389)
    at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3719)
    at org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:2987)
    at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3710)
    at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
    at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
    at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
    at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:774)
    at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
    at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3708)
    at org.apache.spark.sql.Dataset.collect(Dataset.scala:2987)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:299)
    ... 16 more
  Caused by: java.lang.ClassNotFoundException: scala.collection.parallel.TaskSupport
    at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
    at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
    at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352)
    at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
    ... 40 more (ThriftServerQueryTestSuite.scala:209)
```

**After**

```
Run completed in 29 minutes, 17 seconds.
Total number of tests run: 535
Suites: completed 20, aborted 0
Tests: succeeded 535, failed 0, canceled 0, ignored 17, pending 0
All tests passed.
```

Closes #32994 from LuciferYang/SPARK-35838.

Authored-by: YangJie <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-22 06:31:24 -07:00
Angerszhuuuu 5a510cf578 [SPARK-35726][SPARK-35769][SQL][FOLLOWUP] Call periodToMonths and durationToMicros in HiveResult should add endField
### What changes were proposed in this pull request?
When we call periodToMonths and durationToMicros  with certain type field, we should pass endField parameter.

### Why are the changes needed?
When we call periodToMonths and durationToMicros  with certain type field, we should pass endField parameter.

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

### How was this patch tested?
Existed UT

Closes #32984 from AngersZhuuuu/SPARK-35726-35769.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-22 11:15:35 +03:00
gengjiaan 43cd6ca687 [SPARK-35378][SQL][FOLLOWUP] isLocal should consider CommandResult
### What changes were proposed in this pull request?
#32513 added the case class `CommandResult` so as we can eagerly execute command locally. But we forgot to update
`isLocal` of `Dataset`.

### Why are the changes needed?
`Dataset.isLocal` should consider `CommandResult`.

### Does this PR introduce _any_ user-facing change?
Yes. If the SQL plan is `CommandResult`, `Dataset.isLocal` must return true.

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

Closes #32963 from beliefer/SPARK-35378-followup2.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-22 07:39:54 +00:00
Venki Korukanti d4d11cfbfb [SPARK-35799][SS] Fix the allUpdatesTimeMs metric measuring in FlatMapGroupsWithStateExec
### What changes were proposed in this pull request?

Fix how we measure the metric `allUpdatesTimeMs` in `FlatMapGroupsWithStateExec` similar to other streaming stateful operators.

### Why are the changes needed?

Metric `allUpdatesTimeMs` meant to capture the start to end walltime of the operator `FlatMapGroupsWithStateExec`, but currently it just [captures](79362c4efc/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala (L121)) the iterator creation time.

Fix it to measure similar to how other stateful operators measure. Example one [here](79362c4efc/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala (L406)). This measurement is not perfect due to the nature of the lazy iterator and also includes the time the consumer operator spent in processing the current operator output, but it should give a good signal when comparing the metric in one microbatch to the metric in another microbatch.

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

No

### How was this patch tested?

Existing UTs for regression. Due to the nature of metric type (time), it is hard to write a UT, but have manually verified.

Closes #32952 from vkorukanti/SPARK-35799.

Authored-by: Venki Korukanti <venki.korukanti@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-06-22 13:57:21 +09:00
William Hyun 89dbf514f5 [SPARK-35850][BUILD] Upgrade scala-maven-plugin to 4.5.3
### What changes were proposed in this pull request?
This PR aims to upgrade the scala-maven-plugin version to 4.5.3.

### Why are the changes needed?
This will upgrade `sbt-compiler-bridge` from 1.3.1 to 1.5.5 in order to bring the latest bug fixes.

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

### How was this patch tested?
Pass the CIs.

Closes #33007 from williamhyun/scalamvnplugin.

Authored-by: William Hyun <william@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-21 21:35:42 -07:00
Takuya UESHIN a8fdb98ecb [SPARK-35470][PYTHON] Fix disallow_untyped_defs mypy checks for pyspark.pandas.base
### What changes were proposed in this pull request?

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

### Why are the changes needed?

We should enable more disallow_untyped_defs mypy checks.

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

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

### How was this patch tested?

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

Closes #32968 from ueshin/issues/SPARK-35470/disallow_untyped_defs_base.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-22 11:25:16 +09:00
Kent Yao 9f734978d9 [SPARK-35700][SQL] Read char/varchar orc table with created and written by external systems
### What changes were proposed in this pull request?

The char/varchar type should be mapped to orc's string type too, see https://orc.apache.org/docs/types.html

### Why are the changes needed?

fix a regression

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

no

### How was this patch tested?

new tests

Closes #33001 from yaooqinn/SPARK-35700.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-21 19:20:55 -07:00
Kevin Su 765106cb80 [SPARK-35699][K8S] Improve error message when creating k8s pod failed
### What changes were proposed in this pull request?

Improve error message when clients use wrong master URL to submit a job to k8s.

### Why are the changes needed?

Current error messages are not clear for users.
```
(base) ➜ spark git:(master) ./bin/spark-submit \
--master k8s://https://192.168.49.3:8443 \
--name spark-pi \
--class org.apache.spark.examples.SparkPi \
--conf spark.executor.instances=3 \
--conf spark.kubernetes.authenticate.driver.serviceAccountName=spark \
--conf spark.kubernetes.container.image=pingsutw/spark:testing \
local:///opt/spark/examples/jars/spark-examples_2.12-3.2.0-SNAPSHOT.jar
21/06/09 20:50:37 WARN Utils: Your hostname, kobe-pc resolves to a loopback address: 127.0.1.1; using 192.168.103.20 instead (on interface ens160)
21/06/09 20:50:37 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
21/06/09 20:50:38 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
21/06/09 20:50:38 INFO SparkKubernetesClientFactory: Auto-configuring K8S client using current context from users K8S config file
21/06/09 20:50:39 INFO KerberosConfDriverFeatureStep: You have not specified a krb5.conf file locally or via a ConfigMap. Make sure that you have the krb5.conf locally on the driver image.
Exception in thread "main" io.fabric8.kubernetes.client.KubernetesClientException: Operation: [create] for kind: [Pod] with name: [null] in namespace: [default] failed.
at io.fabric8.kubernetes.client.KubernetesClientException.launderThrowable(KubernetesClientException.java:64) at io.fabric8.kubernetes.client.KubernetesClientException.launderThrowable(KubernetesClientException.java:64)
at io.fabric8.kubernetes.client.KubernetesClientException.launderThrowable(KubernetesClientException.java:72) at io.fabric8.kubernetes.client.KubernetesClientException.launderThrowable(KubernetesClientException.java:72)
at io.fabric8.kubernetes.client.dsl.base.BaseOperation.create(BaseOperation.java:380) at io.fabric8.kubernetes.client.dsl.base.BaseOperation.create(BaseOperation.java:380)
at io.fabric8.kubernetes.client.dsl.base.BaseOperation.create(BaseOperation.java:86) at io.fabric8.kubernetes.client.dsl.base.BaseOperation.create(BaseOperation.java:86)
```
Below command to reproduce;
```
./bin/spark-submit \
  --master k8s://https://192.168.49.2:8443 \
  --deploy-mode cluster \
  --name spark-pi \
  --class org.apache.spark.examples.SparkPi \
  --conf spark.executor.instances=3 \
  --conf spark.kubernetes.authenticate.driver.serviceAccountName=spark \
  --conf spark.kubernetes.container.image=pingsutw/spark:testing \
  local:///opt/spark/examples/jars/spark-examples_2.12-3.2.0-SNAPSHOT.jar
```

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

Yes, users will see more clear error messages.

### How was this patch tested?

Pass the CIs.

Closes #32874 from pingsutw/SPARK-35699.

Authored-by: Kevin Su <pingsutw@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-21 19:10:11 -07:00
Gengliang Wang 2bdd9fe5e3 [SPARK-35839][SQL] New SQL function: to_timestamp_ntz
### What changes were proposed in this pull request?

Implement new SQL function: `to_timestamp_ntz`.
The syntax is similar to the built-in function `to_timestamp`:
```
to_timestamp_ntz ( <date_expr> )

to_timestamp_ntz ( <timestamp_expr> )

to_timestamp_ntz ( <string_expr> [ , <format> ] )
```

The naming is from snowflake: https://docs.snowflake.com/en/sql-reference/functions/to_timestamp.html

### Why are the changes needed?

Adds a new SQL function to create a literal/column of timestamp without time zone.
It's convenient for both end-users and developers.

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

Yes, a new SQL function `to_timestamp_ntz`.

### How was this patch tested?

Unit tests

Closes #32995 from gengliangwang/toTimestampNtz.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-22 09:50:48 +08:00
Xinrong Meng 6ca56b01dc [SPARK-35614][PYTHON] Make the conversion to pandas data-type-based for ExtensionDtypes
### What changes were proposed in this pull request?

We propose to
- introduce the Ops class for ExtensionDtypes: `IntegralExtensionOps`, `FractionalExtensionOps`, `StringExtensionOps`
- make the "conversion to pandas" data-type-based for ExtensionDtypes

Non-goal: same arithmetic operation of ExtensionDtypes have different result dtypes between pandas and pandas API on Spark. That should be adjusted in a separated PR if needed.

### Why are the changes needed?

The conversion to pandas includes logic for checking ExtensionDtypes data types and behaving accordingly.
That makes code hard to change or maintain.

Since we have DataTypeOps defined, we are able to dispatch the specific conversion logic to the `ExtensionOps` classes.

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

No.

### How was this patch tested?

Unit tests.

Closes #32910 from xinrong-databricks/datatypeops_pd_ext.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-06-21 13:19:55 -07:00
Chandni Singh 1fe6daa002 [SPARK-35836][SHUFFLE][CORE] Removed the reference to spark.shuffle.push.based.enabled in ShuffleBlockPusherSuite
### What changes were proposed in this pull request?
It is a trivial change to remove the reference to an incorrect configuration for push-based shuffle from a test suite.
Ref: https://github.com/apache/spark/pull/30312
With SPARK-32917, `ShuffleBlockPusher` and its test suite was introduced. `ShuffleBlockPusher` is created only when push-based shuffle is enabled and the tests in `ShuffleBlockPusherSuite` are just testing the functionality in the pusher. So there is no need to have these configs enabled in these test.

### Why are the changes needed?
This change removes an incorrect configuration from the test suite.

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

### How was this patch tested?
This change just removes an incorrect configuration from the test suite so haven't added any UTs for it.

Closes #32992 from otterc/SPARK-35836.

Authored-by: Chandni Singh <singh.chandni@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-06-21 13:57:55 -05:00
Dongjoon Hyun 0f25cabbc2 [SPARK-35844][INFRA] Add hadoop-cloud profile to PUBLISH_PROFILES
### What changes were proposed in this pull request?

This PR aims to add `hadoop-cloud` profile to `PUBLISH_PROFILES` in order to publish `hadoop-cloud` module.

Note that this doesn't change `BASE_RELEASE_PROFILES` and there is no change in the binary distributions.

### Why are the changes needed?

This is discussed here.
- https://lists.apache.org/thread.html/rf87d755460d5ed85c7b6ac0edad48f53c929a2cd287f30be24afd2ad%40%3Cuser.spark.apache.org%3E

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

Yes, this will provide `hadoop-cloud` module in Maven Central.

### How was this patch tested?

N/A (After merging this, we can check the daily snapshot result)

Closes #33003 from dongjoon-hyun/SPARK-35844.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-21 11:56:21 -07:00
Dongjoon Hyun 2b9902d26a [SPARK-35831][YARN][TEST-MAVEN] Handle PathOperationException in copyFileToRemote on the same src and dest
### What changes were proposed in this pull request?

This PR aims to be more robust on the underlying Hadoop library changes. Apache Spark's `copyFileToRemote` has an option, `force`, to invoke copying always and it can hit `org.apache.hadoop.fs.PathOperationException` in some Hadoop versions.

From Apache Hadoop 3.3.1, we reverted [HADOOP-16878](https://issues.apache.org/jira/browse/HADOOP-16878) as the last revert commit on `branch-3.3.1`. However, it's still in Apache Hadoop 3.4.0.
- a3b9c37a39

### Why are the changes needed?

Currently, Apache Spark Jenkins hits a flakiness issue.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2/lastCompletedBuild/testReport/org.apache.spark.deploy.yarn/ClientSuite/distribute_jars_archive/history/
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/2459/testReport/junit/org.apache.spark.deploy.yarn/ClientSuite/distribute_jars_archive/

```
org.apache.hadoop.fs.PathOperationException:
`Source (file:/home/jenkins/workspace/spark-master-test-maven-hadoop-3.2/resource-managers/yarn/target/tmp/spark-703b8e99-63cc-4ba6-a9bc-25c7cae8f5f9/testJar9120517778809167117.jar) and destination (/home/jenkins/workspace/spark-master-test-maven-hadoop-3.2/resource-managers/yarn/target/tmp/spark-703b8e99-63cc-4ba6-a9bc-25c7cae8f5f9/testJar9120517778809167117.jar)
are equal in the copy command.': Operation not supported
at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:403)
```

Apache Spark has three cases.
- `!compareFs(srcFs, destFs)`: This is safe because we will not have this exception.
- `"file".equals(srcFs.getScheme)`: This is safe because this cannot be a `false` alarm.
- `force=true`:
    - For the `good` alarm part, Spark works in the same way.
    - For the `false` alarm part, Spark is safe because we use `force = true` only for copying `localConfArchive` instead of a general copy between two random clusters.

```scala
val localConfArchive = new Path(createConfArchive(confsToOverride).toURI())
copyFileToRemote(destDir, localConfArchive, replication, symlinkCache, force = true,
destName = Some(LOCALIZED_CONF_ARCHIVE))
```

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

No. This preserves the previous Apache Spark behavior.

### How was this patch tested?

Pass the Jenkins with Maven.

Closes #32983 from dongjoon-hyun/SPARK-35831.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-21 23:28:27 +08:00
Gengliang Wang a100a0186a [SPARK-35842][INFRA] Ignore all .idea folders
### What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/32337, all the `.idea/` in submodules are treated as git difference again.
For example, when I open the project `resource-managers/yarn/` with IntelliJ, the git status becomes
```
Untracked files:
  (use "git add <file>..." to include in what will be committed)
	resource-managers/yarn/.idea/
```
The same issue happens on opening `sql/hive-thriftserver/` with IntelliJ.
We should ignore all the ".idea" directories instead of the one under the root path.

### Why are the changes needed?

Make it more friendly for developers who using IDEA for the development of submodules like yarn, hive-thriftserver, etc.

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

No

### How was this patch tested?

Just infra changes.

Closes #32998 from gengliangwang/improveIgnore.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-21 22:07:02 +08:00
Kousuke Saruta 2c91672259 [SPARK-35775][SQL][TESTS] Check all year-month interval types in aggregate expressions
### What changes were proposed in this pull request?

This PR adds test to check `sum` and `avg` works with all the `YearMonthInterval` types.

### Why are the changes needed?

To ensure the results of aggregations are what is expected.

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

No.

### How was this patch tested?

New test.

Closes #32988 from sarutak/check-interval-agg-ym.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-21 16:47:29 +03:00
Vasily Kolpakov 844f10c742 [SPARK-35391] Fix memory leak in ExecutorAllocationListener
### What changes were proposed in this pull request?
This PR fixes a memory leak in ExecutorAllocationListener.

### Why are the changes needed?
Dynamic allocation stops working under high load (~100 tasks/s, ~5 stages/s) in long-lived (~10 days) spark applications. This PR addresses the problem.

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

### How was this patch tested?
Manual tests. The patch fixed dynamic allocation in production cluster.

Closes #32526 from VasilyKolpakov/SPARK-35391_fix_ExecutorAllocationListener.

Authored-by: Vasily Kolpakov <vasilykolpakov@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2021-06-21 08:23:20 -05:00
tanel.kiis@gmail.com f80be4187e [SPARK-34565][SQL] Collapse Window nodes with Project between them
### What changes were proposed in this pull request?

Extend the `CollapseWindow` rule to collapse `Window` nodes, that have `Project` between them.

### Why are the changes needed?

The analyzer will turn a `dataset.withColumn("colName", expressionWithWindowFunction)` method call to a `Project - Window - Project` chain in the logical plan. When this method is called multiple times in a row, then the projects can block the `Window` nodes from being collapsed by the current `CollapseWindow` rule.

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

No

### How was this patch tested?

UT

Closes #31677 from tanelk/SPARK-34565_collapse_windows.

Lead-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Co-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-06-21 22:10:49 +09:00
Max Gekk 37ef7bb98c [SPARK-35840][SQL] Add apply() for a single field to YearMonthIntervalType and DayTimeIntervalType
### What changes were proposed in this pull request?
In the PR, I propose to add 2 new methods that accept one field and produce either `YearMonthIntervalType` or `DayTimeIntervalType`.

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

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

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

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

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

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

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

### How was this patch tested?
Added UT

Closes #32975 from AngersZhuuuu/SPARK-35820.

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

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

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

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

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

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

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

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

Doc changes are following:

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

### Why are the changes needed?

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

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

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

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

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

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

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

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

### How was this patch tested?

New UTs.

Closes #32747 from HeartSaVioR/SPARK-35611.

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

Add a test.

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Added a new test.

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Manually tested. CI should test it out too.

Closes #32989 from HyukjinKwon/SPARK-35834.

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

Deprecate the `DataFrame.to_spark_io`

### Why are the changes needed?

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

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

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

### How was this patch tested?

Pass the CIs

Closes #32964 from pingsutw/SPARK-35811.

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

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

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

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

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

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

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

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

Closes #32811 from otterc/SPARK-35671.

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

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

### Why are the changes needed?

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

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

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

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

No.

### How was this patch tested?

Modified an existing test.

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

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

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

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

### Why are the changes needed?

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

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

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

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

No, this is a test-only change.

### How was this patch tested?

Pass the CIs.

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

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

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

### Why are the changes needed?

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

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

No, because this feature is not released yet.

### How was this patch tested?

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

Closes #32982 from sarutak/fix-toYearMonthIntervalString.

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

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

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

### Why are the changes needed?

To mimic the JVM behaviour about thread lifecycle

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

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

### How was this patch tested?

Existing tests should cover this.

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

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

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

### Why are the changes needed?

We should enable more disallow_untyped_defs mypy checks.

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

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

### How was this patch tested?

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

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

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

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

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

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

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

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

### How was this patch tested?
Added UT

Closes #32974 from AngersZhuuuu/SPARK-35819.

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

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

### Why are the changes needed?

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

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

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

No.

### How was this patch tested?

Pass the CIs.

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

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

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Pass the Jenkins.

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

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

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Jenkins test

Closes #32961 from gengliangwang/increaseXss.

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

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

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

### How was this patch tested?
Added UT

Closes #32950 from AngersZhuuuu/SPARK-35726.

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

This PR aims to upgrade SBT to 1.5.4.

### Why are the changes needed?

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

This will bring the latest bug fixes like the following.

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

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

No.

### How was this patch tested?

Pass the CIs.

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

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Pass the CIs

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

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

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

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

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

Closes #32859 from Yikun/SPARK-XXXXX1.

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

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

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

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

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

Closes #32948 from toujours33/SPARK-35796.

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

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

### Why are the changes needed?

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

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

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

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

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

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

### How was this patch tested?

Unit tests.

Closes #32702 from viirya/ignore-metadata.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-06-19 08:20:58 +09:00