Commit graph

8135 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
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
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
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
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
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
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
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
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
Cheng Su 4a34db9a17 [SPARK-32709][SQL] Support writing Hive bucketed table (Parquet/ORC format with Hive hash)
### What changes were proposed in this pull request?

This is a re-work of https://github.com/apache/spark/pull/30003, here we add support for writing Hive bucketed table with Parquet/ORC file format (data source v1 write path and Hive hash as the hash function). Support for Hive's other file format will be added in follow up PR.

The changes are mostly on:

* `HiveMetastoreCatalog.scala`: When converting hive table relation to data source relation, pass bucket info (BucketSpec) and other hive related info as options into `HadoopFsRelation` and `LogicalRelation`, which can be later accessed by `FileFormatWriter` to customize bucket id and file name.

* `FileFormatWriter.scala`: Use `HiveHash` for `bucketIdExpression` if it's writing to Hive bucketed table. In addition, Spark output file name should follow Hive/Presto/Trino bucketed file naming convention. Introduce another parameter `bucketFileNamePrefix` and it introduces subsequent change in `FileFormatDataWriter`.

* `HadoopMapReduceCommitProtocol`: Implement the new file name APIs introduced in https://github.com/apache/spark/pull/33012, and change its sub-class `PathOutputCommitProtocol`, to make Hive bucketed table writing work with all commit protocol (including S3A commit protocol).

### Why are the changes needed?

To make Spark write other-SQL-engines-compatible bucketed table. Currently Spark bucketed table cannot be leveraged by other SQL engines like Hive and Presto, because it uses a different hash function (Spark murmur3hash) and different file name scheme. With this PR, the Spark-written-Hive-bucketed-table can be efficiently read by Presto and Hive to do bucket filter pruning, join, group-by, etc. This was and is blocking several companies (confirmed from Facebook, Lyft, etc) migrate bucketing workload from Hive to Spark.

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

Yes, any Hive bucketed table (with Parquet/ORC format) written by Spark, is properly bucketed and can be efficiently processed by Hive and Presto/Trino.

### How was this patch tested?

* Added unit test in BucketedWriteWithHiveSupportSuite.scala, to verify bucket file names and each row in each bucket is written properly.
* Tested by Lyft Spark team (Shashank Pedamallu) to read Spark-written bucketed table from Trino, Spark and Hive.

Closes #33432 from c21/hive-bucket-v1.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-17 14:28:51 +08:00
Josh Rosen 3ae6e6775b [SPARK-36774][CORE][TESTS] Move SparkSubmitTestUtils to core module and use it in SparkSubmitSuite
### What changes were proposed in this pull request?

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

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

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

### Why are the changes needed?

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

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

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

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

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

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

No.

### How was this patch tested?

I manually ran the affected test suites.

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

Authored-by: Josh Rosen <joshrosen@databricks.com>
Signed-off-by: Josh Rosen <joshrosen@databricks.com>
2021-09-16 14:28:47 -07:00
Thejdeep Gudivada 23f4a650ea [SPARK-36433][WEBUI] Fix log message in WebUI
### What changes were proposed in this pull request?

This fixes the info log message output when starting a WebUI server

### Why are the changes needed?
This is needed by the user to go to the right location of the started service
```
21/08/05 14:33:30 INFO HistoryServer: Bound HistoryServer to 0.0.0.0, and started at http://tgudivad-mn1.test.biz:18080
```

### Does this PR introduce _any_ user-facing change?
Yes, fixes the URL displayed in the logs when starting the service.

### How was this patch tested?
Tested by running an instance of HistoryServer

Closes #33659 from thejdeep/SPARK-36433.

Authored-by: Thejdeep Gudivada <tgudivada@linkedin.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-09-16 17:26:56 +08:00
yangjie01 119ddd7e95 [SPARK-36737][BUILD][CORE][SQL][SS] Upgrade Apache commons-io to 2.11.0 and revert change of SPARK-36456
### What changes were proposed in this pull request?
SPARK-36456 change to use `JavaUtils. closeQuietly` instead of `IOUtils.closeQuietly`, but there is slightly different from the 2 methods in default behavior: swallowing IOException is same, but the former logs it as ERROR while the latter doesn't log by default.

`Apache commons-io` community decided to retain the `IOUtils.closeQuietly` method in the [new version](75f20dca72/src/main/java/org/apache/commons/io/IOUtils.java (L465-L467)) and removed deprecated annotation,  the change has been released in version 2.11.0.

So the change of this pr is to upgrade `Apache commons-io` to 2.11.0 and revert change of SPARK-36456 to maintain original behavior(don't print error log).

### Why are the changes needed?

1. Upgrade `Apache commons-io` to 2.11.0 to use non-deprecated `closeQuietly` API, other changes related to `Apache commons-io are detailed in [commons-io/changes-report](https://commons.apache.org/proper/commons-io/changes-report.html#a2.11.0)

2. Revert change of SPARK-36737 to maintain original `IOUtils.closeQuietly` API behavior(don't print error log).

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #33977 from LuciferYang/upgrade-commons-io.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-09-14 21:16:58 +09:00
Minchu Yang 2d7dc7c7ce [SPARK-36705][FOLLOW-UP] Fix unnecessary logWarning when PUSH_BASED_SHUFFLE_ENABLED is set to false
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Passed existing UT.

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

Authored-by: Minchu Yang <minyang@minyang-mn3.linkedin.biz>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-09-13 23:23:33 -05:00
Minchu Yang 999473b1a5 [SPARK-36705][SHUFFLE] Disable push based shuffle when IO encryption is enabled or serializer is not relocatable
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

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

Closes #33976 from rmcyang/SPARK-36705.

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

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

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

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

### Why are the changes needed?

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

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

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

No

### How was this patch tested?

Locally

Closes #33948 from lrytz/parCollDep.

Authored-by: Lukas Rytz <lukas.rytz@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-09-13 11:06:50 -05:00
yangjie01 0e1157df06 [SPARK-36636][CORE][TEST] LocalSparkCluster change to use tmp workdir in test to avoid directory name collision
### What changes were proposed in this pull request?
As described in SPARK-36636,if the test cases with config `local-cluster[n, c, m]`  are run continuously within 1 second, the workdir name collision will occur because appid use format as `app-yyyyMMddHHmmss-0000` and workdir name associated with it  in test now,  the related logs are as follows:

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

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

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

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

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

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

### How was this patch tested?

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

**Before**

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

```

**After**

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

Closes #33963 from LuciferYang/SPARK-36636.

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

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

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

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

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

Authored-by: dgd_contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-11 15:58:31 -07:00
yi.wu 7103a165d5 Revert "[SPARK-35011][CORE] Avoid Block Manager registrations when StopExecutor msg is in-flight"
This reverts commit b9e53f8937.

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Pass existing tests.

Closes #33942 from Ngone51/SPARK-36700.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-09-09 20:21:14 -07:00
dgd-contributor 711577e238 [SPARK-36687][SQL][CORE] Rename error classes with _ERROR suffix
### What changes were proposed in this pull request?
redundant _ERROR suffix in error-classes.json

### Why are the changes needed?
Clean up error classes  to reduce clutter

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

### How was this patch tested?
Existing tests

Closes #33944 from dgd-contributor/SPARK-36687.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-10 10:00:28 +09:00
Liang-Chi Hsieh 6bcf330191 [SPARK-36669][SQL] Add Lz4 wrappers for Hadoop Lz4 codec
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

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

No

### How was this patch tested?

Modified test.

Closes #33940 from viirya/lz4-wrappers.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-09 09:31:00 -07:00
Angerszhuuuu 67421d80b8 [SPARK-36692][CORE] Improve Error statement when requesting thread dump while executor already stopped
### What changes were proposed in this pull request?
For now, when user check thread dump for a executor while this executor is stopped, the error log shows following might disturb users.

![image](https://user-images.githubusercontent.com/46485123/132471501-db96894d-abe9-4d62-9943-06c578382ef2.png)

### Why are the changes needed?
Improve error message

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

### How was this patch tested?

Closes #33935 from AngersZhuuuu/SPARK-36692.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-09 10:04:24 +09:00
Liang-Chi Hsieh 6745d77818 [SPARK-36682][CORE][TEST] Add Hadoop sequence file test for different Hadoop codecs
### What changes were proposed in this pull request?

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

### Why are the changes needed?

To improve test coverage.

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

No

### How was this patch tested?

Added tests.

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

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-07 13:19:58 -07:00
yangjie01 35848385ae [SPARK-36602][COER][SQL] Clean up redundant asInstanceOf casts
### What changes were proposed in this pull request?
The change of this pr is remove redundant asInstanceOf casts in Spark code.

### Why are the changes needed?
Code simplification

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

### How was this patch tested?

- Pass GA or Jenkins Tests.

Closes #33852 from LuciferYang/cleanup-asInstanceof.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-09-05 08:22:28 -05:00
yi.wu ebe7bb6217 [SPARK-36614][CORE][UI] Correct executor loss reason caused by decommission in UI
### What changes were proposed in this pull request?

Post the correct executor loss reason to UI.

### Why are the changes needed?

To show the accurate loss reason.

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

Yes. Users can see the difference from the UI.

Before:
<img width="509" alt="WeChataad8d1f27d9f9aa7cf93ced4bcc820e2" src="https://user-images.githubusercontent.com/16397174/131341692-6f412607-87b8-405e-822d-0d28f07928da.png">
<img width="1138" alt="WeChat13c9f1345a096ff83d193e4e9853b165" src="https://user-images.githubusercontent.com/16397174/131341699-f2c9de09-635f-49df-8e27-2495f34276c0.png">

After:

<img width="599" alt="WeChata4313fa2dbf27bf2dbfaef5c1d4a19cf" src="https://user-images.githubusercontent.com/16397174/131341754-e3c93b5d-5252-4006-a4cc-94d76f41303b.png">
<img width="1182" alt="WeChat5559d52fd3070ae6c42fe32d56f9dc94" src="https://user-images.githubusercontent.com/16397174/131341761-e1e0644f-1e76-49c0-915a-26aad77ec272.png">

### How was this patch tested?

Manully tested.

Closes #33868 from Ngone51/fix-executor-remove-reason.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-08-30 09:09:22 -07:00
PengLei df3e5a620a [SPARK-36336][SQL][FOLLOWUP][SPARK] Mark exception private
### What changes were proposed in this pull request?

Mark the exception added `private[spark]`
according [comments](https://github.com/apache/spark/pull/33573#discussion_r696324962)

### Why are the changes needed?
[comments](https://github.com/apache/spark/pull/33573#discussion_r696324962)

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

### How was this patch tested?
existed ut testcase

Closes #33856 from Peng-Lei/SPARK-36336-FOLLOW.

Authored-by: PengLei <peng.8lei@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-30 10:08:44 +09:00
yangjie01 9cefde8db3 [SPARK-36580][CORE][K8S] Use intersect and diff API on Set instead of manual implementation
### What changes were proposed in this pull request?
The main change of this pr is replace `filter` + `contains` with `intersect` api and `filterNot` + `contains` with `diff`

**Before**

```scala
val set = Set(1, 2)
val others = Set(2, 3)
set.filter(others.contains(_))
set.filterNot(others.contains)
```

**After**
```scala
val set = Set(1, 2)
val others = Set(2, 3)
set.intersect(others)
set.diff(others)
```

### Why are the changes needed?
Code simplification, replace manual implementation with existing API

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #33829 from LuciferYang/SPARK-36580.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-29 09:24:37 -07:00
Kousuke Saruta ea8c31e5ea [SPARK-36509][CORE] Fix the issue that executors are never re-scheduled if the worker stops with standalone cluster
### What changes were proposed in this pull request?

This PR fixes an issue that executors are never re-scheduled if the worker which the executors run on stops.
As a result, the application stucks.
You can easily reproduce this issue by the following procedures.

```
# Run master
$ sbin/start-master.sh

# Run worker 1
$ SPARK_LOG_DIR=/tmp/worker1 SPARK_PID_DIR=/tmp/worker1/ sbin/start-worker.sh -c 1 -h localhost -d /tmp/worker1 --webui-port 8081 spark://<hostname>:7077

# Run worker 2
$ SPARK_LOG_DIR=/tmp/worker2 SPARK_PID_DIR=/tmp/worker2/ sbin/start-worker.sh -c 1 -h localhost -d /tmp/worker2 --webui-port 8082 spark://<hostname>:7077

# Run Spark Shell
$ bin/spark-shell --master spark://<hostname>:7077 --executor-cores 1 --total-executor-cores 1

# Check which worker the executor runs on and then kill the worker.
$ kill <worker pid>
```

With the procedure above, we will expect that the executor is re-scheduled on the other worker but it won't.

The reason seems that `Master.schedule` cannot be called after the worker is marked as `WorkerState.DEAD`.
So, the solution this PR proposes is to call `Master.schedule` whenever `Master.removeWorker` is called.

This PR also fixes an issue that `ExecutorRunner` can send `ExecutorStateChanged` message without changing its state.
This issue causes assertion error.
```
2021-08-13 14:05:37,991 [dispatcher-event-loop-9] ERROR: Ignoring errorjava.lang.AssertionError: assertion failed: executor 0 state transfer from RUNNING to RUNNING is illegal
```

### Why are the changes needed?

It's a critical bug.

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

No.

### How was this patch tested?

Manually tested with the procedure shown above and confirmed the executor is re-scheduled.

Closes #33818 from sarutak/fix-scheduling-stuck.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-08-28 18:01:55 +09:00
Gengliang Wang e650d06ba9 [SPARK-36597][DOCS] Fix issues in SQL function docs
### What changes were proposed in this pull request?

* the functions make_dt_interval and make_ym_interval should make it clear that some of the fields are optional
* remove the `|` symbol from the doc of `bit_get` https://dist.apache.org/repos/dist/dev/spark/v3.2.0-rc1-docs/_site/api/sql/index.html#bit_get
* Address one missing comment in https://github.com/apache/spark/pull/33824#discussion_r695405699

### Why are the changes needed?

Improve the documentation.

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

No

### How was this patch tested?

Build doc and preview:
![image](https://user-images.githubusercontent.com/1097932/130996918-8c1fff88-ef5a-434b-8445-df7140bad3ba.png)
![image](https://user-images.githubusercontent.com/1097932/130996954-0ced28e7-fb90-4fcc-857e-6ccc31dc3c09.png)

![image](https://user-images.githubusercontent.com/1097932/130955106-5ae32dfc-6e89-4e28-bb8a-6c1b5213051c.png)

![image](https://user-images.githubusercontent.com/1097932/130922351-2f0f262d-5624-4d08-ba83-dfa3ed0b646b.png)

Closes #33847 from gengliangwang/auditSQLDoc.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-08-27 13:29:34 +08:00
Gengliang Wang 1a42aa5bd4 [SPARK-36457][DOCS] Review and fix issues in Scala/Java API docs
### What changes were proposed in this pull request?

Compare the 3.2.0 API doc with the latest release version 3.1.2. Fix the following issues:

- Add missing `Since` annotation for new APIs
- Remove the leaking class/object in API doc

### Why are the changes needed?

Improve API docs

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

No

### How was this patch tested?

Existing UT

Closes #33824 from gengliangwang/auditDoc.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-08-26 12:59:18 +08:00
Holden Karau ff3f3c4566 [SPARK-36058][K8S] Add support for statefulset APIs in K8s
### What changes were proposed in this pull request?

Generalize the pod allocator and add support for statefulsets.

### Why are the changes needed?

Allocating individual pods in Spark can be not ideal for some clusters and using higher level operators like statefulsets and replicasets can be useful.

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

Yes new config options.

### How was this patch tested?

Completed: New unit & basic integration test
PV integration tests

Closes #33508 from holdenk/SPARK-36058-support-replicasets-or-job-api-like-things.

Lead-authored-by: Holden Karau <hkarau@netflix.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@netflix.com>
2021-08-25 17:38:57 -07:00
PengLei 3e32ea17db [SPARK-36336][SQL] Add new exception of base exception used in QueryExecutionErrors
### What changes were proposed in this pull request?
When we refactor the query execution errors to use error classes in QueryExecutionErrors, we need define some exception that mix SparkThrowable into a base Exception type.
according the example [SparkArithmeticException](f90eb6a5db/core/src/main/scala/org/apache/spark/SparkException.scala (L75))

Add SparkXXXException as follows:
- `SparkClassNotFoundException`
- `SparkConcurrentModificationException`
- `SparkDateTimeException`
- `SparkFileAlreadyExistsException`
- `SparkFileNotFoundException`
- `SparkNoSuchMethodException`
- `SparkIndexOutOfBoundsException`
- `SparkIOException`
- `SparkSecurityException`
- `SparkSQLException`
- `SparkSQLFeatureNotSupportedException`

Refactor some exceptions in QueryExecutionErrors to use error classes and new exception for testing new exception

Some added by [PR](https://github.com/apache/spark/pull/33538) as follows:

- `SparkUnsupportedOperationException`
- `SparkIllegalStateException`
- `SparkNumberFormatException`
- `SparkIllegalArgumentException`
- `SparkArrayIndexOutOfBoundsException`
- `SparkNoSuchElementException`

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

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

### How was this patch tested?
existed ut test

Closes #33573 from Peng-Lei/SPARK-36336.

Authored-by: PengLei <peng.8lei@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-25 09:39:28 +09:00
yi.wu d6c453aaea [SPARK-36564][CORE] Fix NullPointerException in LiveRDDDistribution.toApi
### What changes were proposed in this pull request?

This PR fixes `NullPointerException` in `LiveRDDDistribution.toApi`.

### Why are the changes needed?

Looking at the stack trace, the NPE is caused by the null `exec.hostPort`. I can't get the complete log to take a close look but only guess that it might be due to the event `SparkListenerBlockManagerAdded` is dropped or out of order.

```
21/08/23 12:26:29 ERROR AsyncEventQueue: Listener AppStatusListener threw an exception
java.lang.NullPointerException
	at com.google.common.base.Preconditions.checkNotNull(Preconditions.java:192)
	at com.google.common.collect.MapMakerInternalMap.putIfAbsent(MapMakerInternalMap.java:3507)
	at com.google.common.collect.Interners$WeakInterner.intern(Interners.java:85)
	at org.apache.spark.status.LiveEntityHelpers$.weakIntern(LiveEntity.scala:696)
	at org.apache.spark.status.LiveRDDDistribution.toApi(LiveEntity.scala:563)
	at org.apache.spark.status.LiveRDD.$anonfun$doUpdate$4(LiveEntity.scala:629)
	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
	at scala.collection.mutable.HashMap$$anon$2.$anonfun$foreach$3(HashMap.scala:158)
	at scala.collection.mutable.HashTable.foreachEntry(HashTable.scala:237)
	at scala.collection.mutable.HashTable.foreachEntry$(HashTable.scala:230)
	at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:44)
	at scala.collection.mutable.HashMap$$anon$2.foreach(HashMap.scala:158)
	at scala.collection.TraversableLike.map(TraversableLike.scala:238)
	at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
	at scala.collection.AbstractTraversable.map(Traversable.scala:108)
	at org.apache.spark.status.LiveRDD.doUpdate(LiveEntity.scala:629)
	at org.apache.spark.status.LiveEntity.write(LiveEntity.scala:51)
	at org.apache.spark.status.AppStatusListener.update(AppStatusListener.scala:1206)
	at org.apache.spark.status.AppStatusListener.maybeUpdate(AppStatusListener.scala:1212)
	at org.apache.spark.status.AppStatusListener.$anonfun$onExecutorMetricsUpdate$6(AppStatusListener.scala:956)
	...
```

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

Yes, users will see the expected RDD info in UI instead of the NPE error.

 ### How was this patch tested?

Pass existing tests.

Closes #33812 from Ngone51/fix-hostport-npe.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-24 13:33:42 -07:00
Aravind Patnam ee20fbb3dc [SPARK-36419][CORE] Optionally move final aggregation in RDD.treeAggregate to executor
## What changes were proposed in this pull request?

Move final iteration of aggregation of RDD.treeAggregate to an executor with one partition and fetch that result to the driver

## Why are the changes needed?
1. RDD.fold pulls all shuffle partitions to the driver to merge the result
        a. Driver becomes a single point of failure in the case that there are a lot of partitions to do the final aggregation on
2. Shuffle machinery at executors is much more robust/fault tolerant compared to fetching results to driver.

## Does this PR introduce any user-facing change?
The previous behavior always did the final aggregation in the driver. The user can now (optionally) provide a boolean config (default = false) ENABLE_EXECUTOR_TREE_AGGREGATE to do that final aggregation in a single partition executor before fetching the results to the driver. The only additional cost is that the user will see an extra stage in their job.

## How was this patch tested?
This patch was tested via unit tests, and also tested on a cluster.
The screenshots showing the extra stage on a cluster are attached below (before vs after).
![before](https://user-images.githubusercontent.com/24758726/128249830-eefc4bda-f737-4d68-960e-1d1907762538.png)
![after](https://user-images.githubusercontent.com/24758726/128249838-be70bc95-9f39-489c-be17-c9c80c4846a4.png)

Closes #33644 from akpatnam25/SPARK-36419.

Authored-by: Aravind Patnam <apatnam@linkedin.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-08-24 22:29:26 +08:00
Venkata krishnan Sowrirajan 7b2842e986 [SPARK-36374][FOLLOW-UP] Change config key spark.shuffle.server.mergedShuffleFileManagerImpl to spark.shuffle.push.server.mergedShuffleFileManagerImpl
### What changes were proposed in this pull request?

Minor changes to change the config key name from `spark.shuffle.server.mergedShuffleFileManagerImpl` to `spark.shuffle.push.server.mergedShuffleFileManagerImpl`. This is missed out in https://github.com/apache/spark/pull/33615.

### Why are the changes needed?

To keep the config names consistent

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

Yes, this is a change in the config key name. But the new config name changes are yet to be released. Technically there is no user facing change because of this change.

### How was this patch tested?

Existing tests.

Closes #33799 from venkata91/SPARK-36374-follow-up.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-08-22 01:28:31 -05:00
yangjie01 1ccb06ca8c Revert "[SPARK-34309][BUILD][CORE][SQL][K8S] Use Caffeine instead of Guava Cache"
### What changes were proposed in this pull request?
This pr revert the change of SPARK-34309, includes:

- https://github.com/apache/spark/pull/31517
- https://github.com/apache/spark/pull/33772

### Why are the changes needed?

1. No really performance improvement in Spark
2. Added an additional dependency

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #33784 from LuciferYang/revert-caffeine.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-22 09:36:15 +09:00
yangjie01 1859d9bc85 [SPARK-36407][CORE][SQL] Convert int to long to avoid potential integer multiplications overflow risk
### What changes were proposed in this pull request?
The main change of this pr is converting an int literal to a long literal to avoid potential integer multiplications overflow risk.

For example:

**Before**

```java
void f(int i) {
    long val = 65536 * i;
  }
```

**After**

```java
void f(int i) {
    long val = 65536L * i;
  }
```

### Why are the changes needed?
Avoid potential integer multiplications overflow risk

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #33629 from LuciferYang/cast-to-long.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-08-18 11:30:37 -05:00
yi.wu 996551fece [SPARK-36532][CORE] Fix deadlock in CoarseGrainedExecutorBackend.onDisconnected to avoid executor shutdown hang
### What changes were proposed in this pull request?

Instead of exiting the executor within the RpcEnv's thread, exit the executor in a separate thread.

### Why are the changes needed?

The current exit way in `onDisconnected` can cause the deadlock, which has the exact same root cause with https://github.com/apache/spark/pull/12012:

* `onDisconnected` -> `System.exit` are called in sequence in the thread of `MessageLoop.threadpool`
* `System.exit` triggers shutdown hooks and `executor.stop` is one of the hooks.
* `executor.stop` stops the `Dispatcher`, which waits for the `MessageLoop.threadpool`  to shutdown further.
* Thus, the thread which runs `System.exit` waits for hooks to be done, but the `MessageLoop.threadpool` in the hook waits that thread to finish. Finally, this mutual dependence results in the deadlock.

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

Yes, the executor shutdown won't hang.

### How was this patch tested?

Pass existing tests.

Closes #33759 from Ngone51/fix-executor-shutdown-hang.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-08-18 22:46:48 +08:00
sychen a1ecf83f2e [SPARK-36451][BUILD] Ivy skips looking for source and doc pom
### What changes were proposed in this pull request?
Because SPARK-35863 Upgrade Ivy to 2.5.0, it supports skip searching the source and doc pom, but the remote repo will still be queried at present.

### Why are the changes needed?
Can improve the speed of some UT, such as `IsolatedClientLoader#downloadVersion`, no need to find source and doc pom

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

### How was this patch tested?
manual UT

Closes #33678 from cxzl25/SPARK-36451.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-08-18 08:29:51 -05:00
zhuqi-lucas 05cd5f97c3 [SPARK-35548][CORE][SHUFFLE] Handling new attempt has started error message in BlockPushErrorHandler in client
### What changes were proposed in this pull request?
Add a new type of error message in BlockPushErrorHandler which indicates the PushblockStream message is received after a new application attempt has started. This error message should be correctly handled in client without retrying the block push.

### Why are the changes needed?
When we get a block push failure because of the too old attempt, we will not retry pushing the block nor log the exception on the client side.

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

### How was this patch tested?
Add the corresponding unit test.

Closes #33617 from zhuqi-lucas/master.

Authored-by: zhuqi-lucas <821684824@qq.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-08-16 13:58:48 -05:00
Kazuyuki Tanimura 8ee464cd7a [SPARK-32210][CORE] Fix NegativeArraySizeException in MapOutputTracker with large spark.default.parallelism
### What changes were proposed in this pull request?
The current `MapOutputTracker` class may throw `NegativeArraySizeException` with a large number of partitions. Within the serializeOutputStatuses() method, it is trying to compress an array of mapStatuses and outputting the binary data into (Apache)ByteArrayOutputStream . Inside the (Apache)ByteArrayOutputStream.toByteArray(), negative index exception happens because the index is int and overflows (2GB limit) when the output binary size is too large.

This PR proposes two high-level ideas:
  1. Use `org.apache.spark.util.io.ChunkedByteBufferOutputStream`, which has a way to output the underlying buffer as `Array[Array[Byte]]`.
  2. Change the signatures from `Array[Byte]` to `Array[Array[Byte]]` in order to handle over 2GB compressed data.

### Why are the changes needed?
This issue seems to be missed out in the earlier effort of addressing 2GB limitations [SPARK-6235](https://issues.apache.org/jira/browse/SPARK-6235)

Without this fix, `spark.default.parallelism` needs to be kept at the low number. The drawback of setting smaller spark.default.parallelism is that it requires more executor memory (more data per partition). Setting `spark.io.compression.zstd.level` to higher number (default 1) hardly helps.

That essentially means we have the data size limit that for shuffling and does not scale.

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

### How was this patch tested?
Passed existing tests
```
build/sbt "core/testOnly org.apache.spark.MapOutputTrackerSuite"
```
Also added a new unit test
```
build/sbt "core/testOnly org.apache.spark.MapOutputTrackerSuite  -- -z SPARK-32210"
```
Ran the benchmark using GitHub Actions and didn't not observe any performance penalties. The results are attached in this PR
```
core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt
core/benchmarks/MapStatusesSerDeserBenchmark-results.txt
```

Closes #33721 from kazuyukitanimura/SPARK-32210.

Authored-by: Kazuyuki Tanimura <ktanimura@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-16 09:11:39 -07:00
Venkata krishnan Sowrirajan 2270ecf32f [SPARK-36374][SHUFFLE][DOC] Push-based shuffle high level user documentation
### What changes were proposed in this pull request?

Document the push-based shuffle feature with a high level overview of the feature and corresponding configuration options for both shuffle server side as well as client side. This is how the changes to the doc looks on the browser ([img](https://user-images.githubusercontent.com/8871522/129231582-ad86ee2f-246f-4b42-9528-4ccd693e86d2.png))

### Why are the changes needed?

Helps users understand the feature

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

Docs

### How was this patch tested?

N/A

Closes #33615 from venkata91/SPARK-36374.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-08-16 10:24:40 -05:00
Pablo Langa a9ab41ad56 [SPARK-35320][SQL] Align error message for unsupported key types in MapType in Json reader
### What changes were proposed in this pull request?

This PR is related with https://github.com/apache/spark/pull/33525.
The purpose is to align error messages between the function from_json and the Json reader for unsupported key types in MapType.
Current behavior:
```
scala> spark.read.schema(StructType(Seq(StructField("col", MapType(IntegerType, StringType))))).json(Seq("""{"1": "test"}""").toDS()).show
+----+
| col|
+----+
|null|
+----+

```
```
scala> Seq("""{"1": "test"}""").toDF("col").write.json("/tmp/jsontests1234")

scala> spark.read.schema(StructType(Seq(StructField("col", MapType(IntegerType, StringType))))).json("/tmp/jsontests1234").show
+----+
| col|
+----+
|null|
+----+
```
With this change, an AnalysisException with the message `"Input schema $schema can only contain StringType as a key type for a MapType."` wil be thrown

### Why are the changes needed?

It's more consistent to align the behavior

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

Yes, now an Exception will be thrown

### How was this patch tested?

Unit testing, manual testing

Closes #33672 from planga82/feature/spark35320_improve_error_message_reader.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-15 10:31:57 +09:00
zwangsheng 46f56e632f [SPARK-36487][CORE] Modify exit executor log logic
### What changes were proposed in this pull request?
Adjust the log logic of CoarseGrainedExecutorBackend

### Why are the changes needed?
When exit executor with system code 0, coarseGrainedExecutorBackend will print ERROR log.
That doesn't make sense, because it seems to me that executor's normal decommission is not caused by an exception.

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

### How was this patch tested?
local

Closes #33718 from zwangsheng/enhance/exit-executor-log.

Authored-by: zwangsheng <2213335496@qq.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-08-13 11:44:36 -05:00
yi.wu a47ceaf549 [SPARK-32920][CORE][FOLLOW-UP] Fix string interpolator in the log
### What changes were proposed in this pull request?

fix string interpolator

### Why are the changes needed?

To log the correct stage info.

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

No.

### How was this patch tested?

Pass existed tests.

Closes #33738 from Ngone51/fix.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-13 21:44:22 +09:00
Xingbo Jiang ec5f3a17e3 [SPARK-36500][CORE] Fix temp_shuffle file leaking when a task is interrupted
### What changes were proposed in this pull request?

When a task thread is interrupted, the underlying output stream referred by `DiskBlockObjectWriter.mcs` may have been closed, then we get IOException when flushing the buffered data. This breaks the assumption that `revertPartialWritesAndClose()` should not throw exceptions.

To fix the issue, we can catch the IOException in `ManualCloseOutputStream.manualClose()`.

### Why are the changes needed?

Previously the IOException was not captured, thus `revertPartialWritesAndClose()` threw an exception. When this happens, `BypassMergeSortShuffleWriter.stop()` would stop deleting the temp_shuffle files tracked by `partitionWriters`, hens lead to temp_shuffle file leak issues.

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

No, this is an internal bug fix.

### How was this patch tested?

Tested by running a longevity stress test. After the fix, there is no more leaked temp_shuffle files.

Closes #33731 from jiangxb1987/temp_shuffle.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-13 19:25:20 +09:00
dgd-contributor 9b96b705b2 [SPARK-36097][CORE] Grouping exception in core/scheduler
### What changes were proposed in this pull request?
This PR group exception messages in core/src/main/scala/org/apache/spark/scheduler

### 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 #33529 from dgd-contributor/SPARK-36097.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-08-12 15:27:17 +08:00
yangjie01 f7c85b855b [SPARK-36456][CORE][SQL][SS] Clean up compilation warnings related to method closeQuietly in class IOUtils is deprecated
### What changes were proposed in this pull request?
There are some compilation warnings related to `method closeQuietly in class IOUtils is deprecated`,  `Apache commons-io` suggests that we should use the `try-with-resources` statement or handle suppressed exceptions manually.

The main change of this pr is replace  `o.a.commons.io.IOUtils.closeQuietly` with `o.a.s.network.util.JavaUtils.closeQuietly` directly because all original logic is suppressing `IOException`.

### Why are the changes needed?
Clean up compilation warnings related to `method closeQuietly in class IOUtils is deprecated`

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

### How was this patch tested?

- Pass the Jenkins or GitHub Action

Closes #33682 from LuciferYang/closeQuietly.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-10 23:17:57 -07:00
Min Shen 3f09093a21 [SPARK-36378][SHUFFLE] Switch to using RPCResponse to communicate common block push failures to the client
### What changes were proposed in this pull request?

We have run performance evaluations on the version of push-based shuffle committed to upstream so far, and have identified a few places for further improvements:
1. On the server side, we have noticed that the usage of `String.format`, especially when receiving a block push request, has a much higher overhead compared with string concatenation.
2. On the server side, the usage of `Throwables.getStackTraceAsString` in the `ErrorHandler.shouldRetryError` and `ErrorHandler.shouldLogError` has generated quite some overhead.

These 2 issues are related to how we are currently handling certain common block push failures.
We are communicating such failures via `RPCFailure` by transmitting the exception stack trace.
This generates the overhead on both server and client side for creating these exceptions and makes checking the type of failures fragile and inefficient with string matching of exception stack trace.
To address these, this PR also proposes to encode the common block push failure as an error code and send that back to the client with a proper RPC message.

### Why are the changes needed?

Improve shuffle service efficiency for push-based shuffle.
Improve code robustness for handling block push failures.

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

No

### How was this patch tested?

Existing unit tests.

Closes #33613 from Victsm/SPARK-36378.

Lead-authored-by: Min Shen <mshen@linkedin.com>
Co-authored-by: Min Shen <victor.nju@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-08-10 16:46:55 -05:00