### What changes were proposed in this pull request?
Add trailing slashes in URLs of Spark UI pages.
### Why are the changes needed?
When a user accesses a URL without a trailing slash, Spark UI always responds with a 302 redirect to a URL with a trailing slash.
![image](https://user-images.githubusercontent.com/1097932/110072744-1be92380-7d33-11eb-98d4-50df12f59ae3.png)
Adding trailing slash to URLs in UI pages can reduce such unnecessary redirects
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test. It's a very simple change.
Closes#31753 from gengliangwang/reduceRedirect.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR intends to fix a bug of `objects.NewInstance` if a user runs Spark on jdk8u and a given `cls` in `NewInstance` is a deeply-nested inner class, e.g.,.
```
object OuterLevelWithVeryVeryVeryLongClassName1 {
object OuterLevelWithVeryVeryVeryLongClassName2 {
object OuterLevelWithVeryVeryVeryLongClassName3 {
object OuterLevelWithVeryVeryVeryLongClassName4 {
object OuterLevelWithVeryVeryVeryLongClassName5 {
object OuterLevelWithVeryVeryVeryLongClassName6 {
object OuterLevelWithVeryVeryVeryLongClassName7 {
object OuterLevelWithVeryVeryVeryLongClassName8 {
object OuterLevelWithVeryVeryVeryLongClassName9 {
object OuterLevelWithVeryVeryVeryLongClassName10 {
object OuterLevelWithVeryVeryVeryLongClassName11 {
object OuterLevelWithVeryVeryVeryLongClassName12 {
object OuterLevelWithVeryVeryVeryLongClassName13 {
object OuterLevelWithVeryVeryVeryLongClassName14 {
object OuterLevelWithVeryVeryVeryLongClassName15 {
object OuterLevelWithVeryVeryVeryLongClassName16 {
object OuterLevelWithVeryVeryVeryLongClassName17 {
object OuterLevelWithVeryVeryVeryLongClassName18 {
object OuterLevelWithVeryVeryVeryLongClassName19 {
object OuterLevelWithVeryVeryVeryLongClassName20 {
case class MalformedNameExample2(x: Int)
}}}}}}}}}}}}}}}}}}}}
```
The root cause that Kris (rednaxelafx) investigated is as follows (Kudos to Kris);
The reason why the test case above is so convoluted is in the way Scala generates the class name for nested classes. In general, Scala generates a class name for a nested class by inserting the dollar-sign ( `$` ) in between each level of class nesting. The problem is that this format can concatenate into a very long string that goes beyond certain limits, so Scala will change the class name format beyond certain length threshold.
For the example above, we can see that the first two levels of class nesting have class names that look like this:
```
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassName1$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassName1$OuterLevelWithVeryVeryVeryLongClassName2$
```
If we leave out the fact that Scala uses a dollar-sign ( `$` ) suffix for the class name of the companion object, `OuterLevelWithVeryVeryVeryLongClassName1`'s full name is a prefix (substring) of `OuterLevelWithVeryVeryVeryLongClassName2`.
But if we keep going deeper into the levels of nesting, you'll find names that look like:
```
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$2a1321b953c615695d7442b2adb1$$$$ryVeryLongClassName8$OuterLevelWithVeryVeryVeryLongClassName9$OuterLevelWithVeryVeryVeryLongClassName10$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$2a1321b953c615695d7442b2adb1$$$$ryVeryLongClassName8$OuterLevelWithVeryVeryVeryLongClassName9$OuterLevelWithVeryVeryVeryLongClassName10$OuterLevelWithVeryVeryVeryLongClassName11$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$85f068777e7ecf112afcbe997d461b$$$$VeryLongClassName11$OuterLevelWithVeryVeryVeryLongClassName12$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$85f068777e7ecf112afcbe997d461b$$$$VeryLongClassName11$OuterLevelWithVeryVeryVeryLongClassName12$OuterLevelWithVeryVeryVeryLongClassName13$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$85f068777e7ecf112afcbe997d461b$$$$VeryLongClassName11$OuterLevelWithVeryVeryVeryLongClassName12$OuterLevelWithVeryVeryVeryLongClassName13$OuterLevelWithVeryVeryVeryLongClassName14$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$5f7ad51804cb1be53938ea804699fa$$$$VeryLongClassName14$OuterLevelWithVeryVeryVeryLongClassName15$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$5f7ad51804cb1be53938ea804699fa$$$$VeryLongClassName14$OuterLevelWithVeryVeryVeryLongClassName15$OuterLevelWithVeryVeryVeryLongClassName16$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$5f7ad51804cb1be53938ea804699fa$$$$VeryLongClassName14$OuterLevelWithVeryVeryVeryLongClassName15$OuterLevelWithVeryVeryVeryLongClassName16$OuterLevelWithVeryVeryVeryLongClassName17$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$69b54f16b1965a31e88968df1a58d8$$$$VeryLongClassName17$OuterLevelWithVeryVeryVeryLongClassName18$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$69b54f16b1965a31e88968df1a58d8$$$$VeryLongClassName17$OuterLevelWithVeryVeryVeryLongClassName18$OuterLevelWithVeryVeryVeryLongClassName19$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$69b54f16b1965a31e88968df1a58d8$$$$VeryLongClassName17$OuterLevelWithVeryVeryVeryLongClassName18$OuterLevelWithVeryVeryVeryLongClassName19$OuterLevelWithVeryVeryVeryLongClassName20$
```
with a hash code in the middle and various levels of nesting omitted.
The `java.lang.Class.isMemberClass` method is implemented in JDK8u as:
http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/tip/src/share/classes/java/lang/Class.java#l1425
```
/**
* Returns {code true} if and only if the underlying class
* is a member class.
*
* return {code true} if and only if this class is a member class.
* since 1.5
*/
public boolean isMemberClass() {
return getSimpleBinaryName() != null && !isLocalOrAnonymousClass();
}
/**
* Returns the "simple binary name" of the underlying class, i.e.,
* the binary name without the leading enclosing class name.
* Returns {code null} if the underlying class is a top level
* class.
*/
private String getSimpleBinaryName() {
Class<?> enclosingClass = getEnclosingClass();
if (enclosingClass == null) // top level class
return null;
// Otherwise, strip the enclosing class' name
try {
return getName().substring(enclosingClass.getName().length());
} catch (IndexOutOfBoundsException ex) {
throw new InternalError("Malformed class name", ex);
}
}
```
and the problematic code is `getName().substring(enclosingClass.getName().length())` -- if a class's enclosing class's full name is *longer* than the nested class's full name, this logic would end up going out of bounds.
The bug has been fixed in JDK9 by https://bugs.java.com/bugdatabase/view_bug.do?bug_id=8057919 , but still exists in the latest JDK8u release. So from the Spark side we'd need to do something to avoid hitting this problem.
### Why are the changes needed?
Bugfix on jdk8u.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added tests.
Closes#31733 from maropu/SPARK-34607.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Make the "duration" column in standalone mode master UI sorted by numeric duration, hence the column can be sorted by the correct order.
Before changes:
![image](https://user-images.githubusercontent.com/26694233/110025426-f5a49300-7cf4-11eb-86f0-2febade86be9.png)
After changes:
![image](https://user-images.githubusercontent.com/26694233/110025604-33092080-7cf5-11eb-8b34-215688faf56d.png)
### Why are the changes needed?
Fix a UI bug to make the sorting consistent across different pages.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Ran several apps with different durations and verified the duration column on the master page can be sorted correctly.
Closes#31743 from baohe-zhang/SPARK-32924.
Authored-by: Baohe Zhang <baohe.zhang@verizonmedia.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
SPARK-33084 added the ability to use ivy coordinates with `SparkContext.addJar`. PR #29966 claims to mimic Hive behavior although I found a few cases where it doesn't
1) The default value of the transitive parameter is false, both in case of parameter not being specified in coordinate or parameter value being invalid. The Hive behavior is that transitive is [true if not specified](cb2ac3dcc6/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java (L169)) in the coordinate and [false for invalid values](cb2ac3dcc6/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java (L124)). Also, regardless of Hive, I think a default of true for the transitive parameter also matches [ivy's own defaults](https://ant.apache.org/ivy/history/2.5.0/ivyfile/dependency.html#_attributes).
2) The parameter value for transitive parameter is regarded as case-sensitive [based on the understanding](https://github.com/apache/spark/pull/29966#discussion_r547752259) that Hive behavior is case-sensitive. However, this is not correct, Hive [treats the parameter value case-insensitively](cb2ac3dcc6/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java (L122)).
I propose that we be compatible with Hive for these behaviors
### Why are the changes needed?
To make `ADD JAR` with ivy coordinates compatible with Hive's transitive behavior
### Does this PR introduce _any_ user-facing change?
The user-facing changes here are within master as the feature introduced in SPARK-33084 has not been released yet
1. Previously an ivy coordinate without `transitive` parameter specified did not resolve transitive dependency, now it does.
2. Previously an `transitive` parameter value was treated case-sensitively. e.g. `transitive=TRUE` would be treated as false as it did not match exactly `true`. Now it will be treated case-insensitively.
### How was this patch tested?
Modified existing unit tests to test new behavior
Add new unit test to cover usage of `exclude` with unspecified `transitive`
Closes#31623 from shardulm94/spark-34506.
Authored-by: Shardul Mahadik <smahadik@linkedin.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
This is kind of a followup of https://github.com/apache/spark/pull/24033 and https://github.com/apache/spark/pull/30945.
Many of references in `SecurityManager` were introduced from SPARK-1189, and related usages were removed later from https://github.com/apache/spark/pull/24033 and https://github.com/apache/spark/pull/30945. This PR proposes to remove them out.
### Why are the changes needed?
For better readability of codes.
### Does this PR introduce _any_ user-facing change?
No, dev-only.
### How was this patch tested?
Manually complied. GitHub Actions and Jenkins build should test it out as well.
Closes#31636 from HyukjinKwon/SPARK-34520.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
The endpoints of Prometheus metrics are properly marked and documented as an experimental (SPARK-31674). The class `PrometheusServlet` itself is not the part of an API so this PR proposes to remove it.
### Why are the changes needed?
To avoid marking a non-API as an API.
### Does this PR introduce _any_ user-facing change?
No, the class is already `private[spark]`.
### How was this patch tested?
Existing tests should cover.
Closes#31640 from HyukjinKwon/SPARK-34531.
Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Use `ask` instead of `send` to sync the `ExecutorStateChanged` between Worker and Master and retry(up to 5 times) on the failure until the message is successfully handled by the Master. And the Worker would exit itself if the message can not be sent after 5 times retry.
### Why are the changes needed?
If the Worker fails to send ExecutorStateChanged to the Master due to some unexpected errors, e.g., temporary network error, then the Master can't remove the finished executor normally and think the executor is still alive. In the worst case, if the executor is the only executor for the application, the application can get hang.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass existing tests.
Closes#31348 from Ngone51/periodically-trigger-master-schedule.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Apache Spark 3.0 introduced `spark.eventLog.compression.codec` configuration.
For Apache Spark 3.2, this PR aims to set `zstd` as the default value for `spark.eventLog.compression.codec` configuration.
This only affects creating a new log file.
### Why are the changes needed?
The main purpose of event logs is archiving. Many logs are generated and occupy the storage, but most of them are never accessed by users.
**1. Save storage resources (and money)**
In general, ZSTD is much smaller than LZ4.
For example, in case of TPCDS (Scale 200) log, ZSTD generates about 3 times smaller log files than LZ4.
| CODEC | SIZE (bytes) |
|---------|-------------|
| LZ4 | 184001434|
| ZSTD | 64522396|
And, the plain file is 17.6 times bigger.
```
-rw-r--r-- 1 dongjoon staff 1135464691 Feb 21 22:31 spark-a1843ead29834f46b1125a03eca32679
-rw-r--r-- 1 dongjoon staff 64522396 Feb 21 22:31 spark-a1843ead29834f46b1125a03eca32679.zstd
```
**2. Better Usability**
We cannot decompress Spark-generated LZ4 event log files via CLI while we can for ZSTD event log files. Spark's LZ4 event log files are inconvenient to some users who want to uncompress and access them.
```
$ lz4 -d spark-d3deba027bd34435ba849e14fc2c42ef.lz4
Decoding file spark-d3deba027bd34435ba849e14fc2c42ef
Error 44 : Unrecognized header : file cannot be decoded
```
```
$ zstd -d spark-a1843ead29834f46b1125a03eca32679.zstd
spark-a1843ead29834f46b1125a03eca32679.zstd: 1135464691 bytes
```
**3. Speed**
The following results are collected by running [lzbench](https://github.com/inikep/lzbench) on the above Spark event log. Note that
- This is not a direct comparison of Spark compression/decompression codec.
- `lzbench` is an in-memory benchmark. So, it doesn't show the benefit of the reduced network traffic due to the small size of ZSTD.
Here,
- To get ZSTD 1.4.8-1 result, `lzbench` `master` branch is used because Spark is using ZSTD 1.4.8.
- To get LZ4 1.7.5 result, `lzbench` `v1.7` branch is used because Spark is using LZ4 1.7.1.
```
Compressor name Compress. Decompress. Compr. size Ratio Filename
memcpy 7393 MB/s 7166 MB/s 1135464691 100.00 spark-a1843ead29834f46b1125a03eca32679
zstd 1.4.8 -1 1344 MB/s 3351 MB/s 56665767 4.99 spark-a1843ead29834f46b1125a03eca32679
lz4 1.7.5 1385 MB/s 4782 MB/s 127662168 11.24 spark-a1843ead29834f46b1125a03eca32679
```
### Does this PR introduce _any_ user-facing change?
- No for the apps which doesn't use `spark.eventLog.compress` because `spark.eventLog.compress` is disabled by default.
- No for the apps using `spark.eventLog.compression.codec` explicitly because this is a change of the default value.
- Yes for the apps using `spark.eventLog.compress` without setting `spark.eventLog.compression.codec`. In this case, previously `spark.io.compression.codec` value was used whose default is `lz4`.
So this JIRA issue, SPARK-34503, is labeled with `releasenotes`.
### How was this patch tested?
Pass the updated UT.
Closes#31618 from dongjoon-hyun/SPARK-34503.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR adds missing docs for ResourceProfile related APIs. Besides, it includes a few minor changes on API:
* ResourceProfileBuilder.build -> ResourceProfileBuilder.builder()
* Provides java specific API `allSupportedExecutorResourcesJList`
* private `ResourceAllocator` since it was mistakenly exposed previously
### Why are the changes needed?
Add missing API docs
### Does this PR introduce _any_ user-facing change?
No, as Apache Spark 3.1 hasn't officially released.
### How was this patch tested?
Updated unit tests due to the signature change of `build()`.
Closes#31496 from Ngone51/resource-profile-api-cleanup.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
This PR is a fix for the JLS 17.5.3 violation identified in
zsxwing's [19/Feb/19 11:47 comment](https://issues.apache.org/jira/browse/SPARK-20977?focusedCommentId=16772277&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16772277) on the JIRA.
### What changes were proposed in this pull request?
- Use a var field to hold the state of the collection accumulator
### Why are the changes needed?
AccumulatorV2 auto-registration of accumulator during readObject doesn't work with final fields that are post-processed outside readObject. As it stands incompletely initialized objects are published to heartbeat thread. This leads to sporadic exceptions knocking out executors which increases the cost of the jobs. We observe such failures on a regular basis https://github.com/NVIDIA/spark-rapids/issues/1522.
### Does this PR introduce _any_ user-facing change?
None
### How was this patch tested?
- this is a concurrency bug that is almost impossible to reproduce as a quick unit test.
- By trial and error I crafted a command https://github.com/NVIDIA/spark-rapids/pull/1688 that reproduces the issue on my dev box several times per hour, with the first occurrence often within a few minutes. After the patch, these Exceptions have not shown up after running overnight for 10+ hours
- existing unit tests in *`AccumulatorV2Suite` and *`LiveEntitySuite`
Closes#31540 from gerashegalov/SPARK-20977.
Authored-by: Gera Shegalov <gera@apache.org>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
This PR tries to support the (non-legacy) delay scheduling for the barrier execution.
The idea is, adding a pending launch tasks list(`barrierPendingLaunchTasks`) in the barrier `TaskSetManager`. And we don't really add those pending launch tasks to the running list and post task start event to the listeners and so on until all tasks in the barrier `TaskSetManager` has been added to `barrierPendingLaunchTasks` after a single round `resourceOffers()`. If there're only partial tasks that are able to launch after a single `rousourceOffers()` round, we'll revert all the assigned resources to those tasks which were added in `barrierPendingLaunchTasks` and clear `barrierPendingLaunchTasks` and wait for the next `resourceOffers()` round. The barrier `TaskSetManager` should be launched finally since we've ensured enough slots before the scheduling.
### Why are the changes needed?
Currently, with delay scheduling enabled for the barrier execution, the application can abort immediately when there're only partial tasks can be launched. This is really bad, especially when the application already completed many stages before the barrier stage. For example, the application may do some ETL jobs before the barrier job(for ML).
After this PR, this scenario should no longer happen.
### Does this PR introduce _any_ user-facing change?
Yes, users will no longer face the `Fail resource offers for barrier stage...` error.
### How was this patch tested?
Added/updated unit tests.
Closes#30650 from Ngone51/barrier-delay-scheduling.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
### What changes were proposed in this pull request?
This PR aims to upgrade Zstd-JNI library to 1.4.8-4 to bring JNI side optimization.
`ZStandardBenchmark` shows that there is no regression in terms of performance and show some improvements.
### Why are the changes needed?
https://github.com/luben/zstd-jni/commits/v1.4.8-4
- be9be47fae
- be51ebade1
- 44ff8b6f95
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the CIs.
Closes#31585 from dongjoon-hyun/SPARK-ZSTD-1.4.8-4.
Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR upgrades Jetty from `9.4.34` to `9.4.36`.
### Why are the changes needed?
CVE-2020-27218 affects currently used Jetty 9.4.34.
https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2020-27218
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Modified existing test and new test which comply with the new version of Jetty.
Closes#31574 from sarutak/upgrade-jetty-9.4.36.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR adds the following metrics to track executor remove reasons during dynamic allocation:
- `numberExecutorsGracefullyDecommissioned`: number of executors which reached the finished decommissioning state and shut itself down cleanly
- `numberExecutorsDecommissionUnfinished`: executors which requested to decommission but they stopped without reaching the finished decommissioning state
- `numberExecutorsKilledByDriver`: executors killed by the driver (requested to stop)
- `numberExecutorsExitedUnexpectedly`: executors exited without driver request
### Why are the changes needed?
For supporting monitoring of dynamic allocation better with these metrics.
### Does this PR introduce _any_ user-facing change?
Yes. The new metrics will be available for monitoring.
### How was this patch tested?
With unit and integration tests.
Finally manually checked the new metrics in jconsole:
<img width="1054" alt="jmx" src="https://user-images.githubusercontent.com/2017933/107458686-de8adf00-6b54-11eb-86f7-41faf2fb638f.png">
Closes#31450 from attilapiros/SPARK-33763-final.
Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
### What changes were proposed in this pull request?
This PR introduces a new protected method in `SparkFunSuite` which is only called when the test failed and can be used to collect logs for failed test. By this PR it is implemented in the Kubernetes tests by `KubernetesSuite` class where it collects all the POD logs and logs them out.
This unfortunately cannot be realized with a simple "after" method as in the "after" method the test outcome is not available.
Moreover this PR removes the `appLocator` as a method argument as `appLocator` is available as a member variable.
### Why are the changes needed?
Currently both the driver and executors logs are lost.
In [developer-tools](https://spark.apache.org/developer-tools.html) there is a hint:
"Getting logs from the pods and containers directly is an exercise left to the reader."
But when the test is executed by Jenkins and a failure happened we really need the POD logs to analyze problem.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
By integration testing. I have checked what would happen if one test fails, the output would be:
```
21/02/14 11:05:34.261 ScalaTest-main-running-KubernetesSuite INFO KubernetesSuite:
===== EXTRA LOGS FOR THE FAILED TEST
21/02/14 11:05:34.278 ScalaTest-main-running-KubernetesSuite INFO KubernetesSuite: BEGIN driver POD log
++ id -u
+ myuid=185
++ id -g
+ mygid=0
+ set +e
++ getent passwd 185
+ uidentry=
+ set -e
+ '[' -z '' ']'
+ '[' -w /etc/passwd ']'
+ echo '185❌185:0:anonymous uid:/opt/spark:/bin/false'
+ SPARK_CLASSPATH=':/opt/spark/jars/*'
+ env
+ grep SPARK_JAVA_OPT_
+ sort -t_ -k4 -n
+ sed 's/[^=]*=\(.*\)/\1/g'
+ readarray -t SPARK_EXECUTOR_JAVA_OPTS
+ '[' -n '' ']'
+ '[' -z ']'
+ '[' -z ']'
+ '[' -n '' ']'
+ '[' -z ']'
+ '[' -z x ']'
+ SPARK_CLASSPATH='/opt/spark/conf::/opt/spark/jars/*'
+ case "$1" in
+ shift 1
+ CMD=("$SPARK_HOME/bin/spark-submit" --conf "spark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS" --deploy-mode client "$")
+ exec /usr/bin/tini -s -- /opt/spark/bin/spark-submit --conf spark.driver.bindAddress=172.17.0.3 --deploy-mode client --properties-file /opt/spark/conf/spark.properties --class org.apache.spark.deploy.PythonRunner local:///opt/spark/tests/decommissioning.py
21/02/14 10:02:28 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Starting decom test
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
21/02/14 10:02:29 INFO SparkContext: Running Spark version 3.2.0-SNAPSHOT
21/02/14 10:02:29 INFO ResourceUtils: ==============================================================
21/02/14 10:02:29 INFO ResourceUtils: No custom resources configured for spark.driver.
21/02/14 10:02:29 INFO ResourceUtils: ==============================================================
...
21/02/14 10:03:17 INFO ShutdownHookManager: Deleting directory /var/data/spark-fa6961ed-a2c1-444c-bfeb-20e63ba0b5cf/spark-ab4b0287-6e24-4b39-837e-9b0b62c1f26f
21/02/14 10:03:17 INFO ShutdownHookManager: Deleting directory /tmp/spark-d6b11e7d-6a03-4a1d-8559-37cb853319bf
21/02/14 11:05:34.279 ScalaTest-main-running-KubernetesSuite INFO KubernetesSuite: END driver POD log
```
Closes#31561 from attilapiros/SPARK-34426.
Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
Lazily load Hive's configuration properties from `hive-site.xml` only once.
### Why are the changes needed?
It is expensive to parse the same file over and over.
### Does this PR introduce _any_ user-facing change?
Should not. The changes can improve performance slightly.
### How was this patch tested?
By existing test suites such as `SparkContextSuite`.
Closes#31556 from MaxGekk/load-hive-site-once.
Authored-by: herman <herman@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Allow users to have Spark attempt to decommission excluded executors.
Since excluded executors may be flaky, this also adds the ability for users to specify a time limit after which a decommissioning executor will be killed by Spark.
### Why are the changes needed?
This may help prevent fetch failures from excluded executors, and also handle the situation in which executors
### Does this PR introduce _any_ user-facing change?
Yes, two new configuration flags for the behaviour.
### How was this patch tested?
Extended unit and integration tests.
Closes#31539 from holdenk/re=enable-SPARK-34104-SPARK-34105.
Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
### What changes were proposed in this pull request?
Allow users to have Spark attempt to decommission excluded executors.
Since excluded executors may be flaky, this also adds the ability for users to specify a time limit after which a decommissioning executor will be killed by Spark.
### Why are the changes needed?
This may help prevent fetch failures from excluded executors, and also handle the situation in which executors
### Does this PR introduce _any_ user-facing change?
Yes, two new configuration flags for the behaviour.
### How was this patch tested?
Extended unit and integration tests.
Closes#31249 from holdenk/configure-inaccessibleList-kill-to-use-decommissioning.
Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@apple.com>
### What changes were proposed in this pull request?
Allow users to configure a maximum amount of shuffle blocks to be stored and reject remote shuffle blocks when this threshold is exceeded.
### Why are the changes needed?
In disk constrained environments with large amount of shuffle data, migrations may result in excessive disk pressure on the nodes. On Kube nodes this can result in cascading failures when combined with `emptyDir`.
### Does this PR introduce _any_ user-facing change?
Yes, new configuration parameter.
### How was this patch tested?
New unit tests.
Closes#31493 from holdenk/SPARK-34337-reject-disk-blocks-when-under-disk-pressure.
Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@apple.com>
### What changes were proposed in this pull request?
When doing https://issues.apache.org/jira/browse/SPARK-34399 based on https://github.com/apache/spark/pull/31471
Found FileBatchWrite will use `FileFormatWrite.processStates()` too. We need log commit duration in other writer too.
In this pr:
1. Extract a commit job method in SparkHadoopWriter
2. address other commit writer
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
No
Closes#31520 from AngersZhuuuu/SPARK-34355-followup.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
The getMetricsSnapshot method of the PrometheusServlet class has a wrong value, It should be taking the mean value but it's taking the max value.
### Why are the changes needed?
The mean value of timersLabels in the PrometheusServlet class is wrong, You can look at line 105 of this class: L105.
```
sb.append(s"${prefix}Mean$timersLabels ${snapshot.getMax}\n")
```
it should be
```
sb.append(s"${prefix}Mean$timersLabels ${snapshot.getMean}\n")
```
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
![image](https://user-images.githubusercontent.com/5170878/107313576-cc199280-6acd-11eb-9384-b6abf71c0f90.png)
Closes#31532 from 397090770/SPARK-34405.
Authored-by: wyp <wyphao.2007@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
`Mockito.initMocks(Object)` is a deprecated api, should use `Mockito.openMocks(Object).close()` instead.
### Why are the changes needed?
Cleanup deprecation api usage.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#31487 from LuciferYang/mockito-api.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This is a follow-up for SPARK-34346 which causes a flakiness due to `core-site.xml` test resource file addition. This PR aims to remove the test resource `core/src/test/resources/core-site.xml` from `core` module.
### Why are the changes needed?
Due to the test resource `core-site.xml`, YARN UT becomes flaky in GitHub Action and Jenkins.
```
$ build/sbt "yarn/testOnly *.YarnClusterSuite -- -z SPARK-16414" -Pyarn
...
[info] YarnClusterSuite:
[info] - yarn-cluster should respect conf overrides in SparkHadoopUtil (SPARK-16414, SPARK-23630) *** FAILED *** (20 seconds, 209 milliseconds)
[info] FAILED did not equal FINISHED (stdout/stderr was not captured) (BaseYarnClusterSuite.scala:210)
```
To isolate more, we may use `SPARK_TEST_HADOOP_CONF_DIR` like `yarn` module's `yarn/Client`, but it seems an overkill in `core` module.
```
// SPARK-23630: during testing, Spark scripts filter out hadoop conf dirs so that user's
// environments do not interfere with tests. This allows a special env variable during
// tests so that custom conf dirs can be used by unit tests.
val confDirs = Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR") ++
(if (Utils.isTesting) Seq("SPARK_TEST_HADOOP_CONF_DIR") else Nil)
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the CIs.
Closes#31515 from dongjoon-hyun/SPARK-34346-2.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR aims to add ZStandardBenchmark as a base-line.
### Why are the changes needed?
This will prevent any regression when we upgrade Zstandard library in the future.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manually.
Closes#31498 from dongjoon-hyun/SPARK-ZSTD-BENCH.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
In many real-world cases, when interacting with hive catalog through Spark SQL, users may just share the `hive-site.xml` for their hive jobs and make a copy to `SPARK_HOME`/conf w/o modification. In Spark, when we generate Hadoop configurations, we will use `spark.buffer.size(65536)` to reset `io.file.buffer.size(4096)`. But when we load the hive-site.xml, we may ignore this behavior and reset `io.file.buffer.size` again according to `hive-site.xml`.
1. The configuration priority for setting Hadoop and Hive config here is not right, while literally, the order should be `spark > spark.hive > spark.hadoop > hive > hadoop`
2. This breaks `spark.buffer.size` congfig's behavior for tuning the IO performance w/ HDFS if there is an existing `io.file.buffer.size` in hive-site.xml
### Why are the changes needed?
bugfix for configuration behavior and fix performance regression by that behavior change
### Does this PR introduce _any_ user-facing change?
this pr restores silent user face change
### How was this patch tested?
new tests
Closes#31460 from yaooqinn/SPARK-34346.
Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR proposes to expose the number of total paths in Utils.buildLocationMetadata(), with relaxing space usage a bit (around 10+ chars).
Suppose the first 2 of 5 paths are only fit to the threshold, the outputs between the twos are below:
* before the change: `[path1, path2]`
* after the change: `(5 paths)[path1, path2, ...]`
### Why are the changes needed?
SPARK-31793 silently truncates the paths hence end users can't indicate how many paths are truncated, and even more, whether paths are truncated or not.
### Does this PR introduce _any_ user-facing change?
Yes, the location metadata will also show how many paths are truncated (not shown), instead of silently truncated.
### How was this patch tested?
Modified UTs
Closes#31464 from HeartSaVioR/SPARK-34339.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR proposes to fix the UTs being added in SPARK-31793, so that all things contributing the length limit are properly accounted.
### Why are the changes needed?
The test `DataSourceScanExecRedactionSuite.SPARK-31793: FileSourceScanExec metadata should contain limited file paths` is failing conditionally, depending on the length of the temp directory.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Modified UTs explain the missing points, which also do the test.
Closes#31449 from HeartSaVioR/SPARK-34326-v2.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
This PR aims two goals.
1. Support ZSTD JNI BufferPool feature by adding a new configuration, `spark.io.compression.zstd.bufferPool.enabled`, for Apache Spark 3.2.0.
2. Make Spark independent from ZSTD JNI library's default buffer pool policy change.
### Why are the changes needed?
ZSTD JNI library has different behaviors across its versions.
| Version | Description | Commit |
| ---------- | --------------- | ----------- |
| v1.4.5-7 | `BufferPool` was added and used it by default | 4f55c89172 |
| v1.4.5-8 | `RecyclingBufferPool` was added and `BufferPool` became an interface to allow custom BufferPool implementation | dd2588edd3 |
| v1.4.7+ | `NoPool` is used by default and user should specify buffer pool explicitly | f7c8279bc1 |
### Does this PR introduce _any_ user-facing change?
No, the default value (`false`) is consistent with the AS-IS ZSTD-JNI library's default buffer pool.
### How was this patch tested?
Pass the CIs with the updated UT.
Closes#31453 from dongjoon-hyun/SPARK-34340.
Lead-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Improve the error messages for incompatibilities between Avro and Catalyst schemas. First, make `AvroSerializer` more similar to `AvroDeserializer` in printing out contextual information such as hierarchical field names. Standardize exception messages in both serializer and deserializer to always include such contextual information, and include a top-level exception which shows the full schemas which were being parsed when the incompatibility was found. Both now print out the hierarchical name for both the Avro and Catalyst fields, since they may be different due to case sensitivity and Avro union handling.
### Why are the changes needed?
The error messages in this type of failure scenario are very lacking in information on the write path (`AvroSerializer`). Below are two examples of messages that provide insufficient information to determine what went wrong (lacking in field names, context about the overall schema structure, etc.).
```
org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Catalyst type IntegerType to Avro type "float".
org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Catalyst type StructType(StructField(bar,IntegerType,true)) to Avro type {"type":"record","name":"test","fields":[{"name":"NOTbar","type":["null","int"],"default":null}]}.
```
The error messages currently existing in `AvroDeserializer` are much better, but still not very internally consistent, and it would be better if they were consistent with the newly added exception messages in `AvroSerializer`.
### Does this PR introduce _any_ user-facing change?
Error messages when there are incompatibilities between Avro and Catalyst schemas will be greatly improved on when writing Avro data using the `avroSchema` option, a little bit improved when reading Avro data, and much more consistent between the two.
Below is an example of a new message. See `AvroSerdeSuite` for more examples.
```
org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Catalyst type STRUCT<`foo`: STRUCT<`bar`: INT>> to Avro type {"type":"record","name":"top","fields":[{"name":"foo","type":"int"}]}
at org.apache.spark.sql.avro.AvroSerializer.liftedTree1$1(AvroSerializer.scala:83)
...
Caused by: org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Catalyst field 'foo' to Avro field 'foo' because schema is incompatible (sqlType = STRUCT<`bar`: INT>, avroType = "int")
at org.apache.spark.sql.avro.AvroSerializer.newConverter(AvroSerializer.scala:230)
...
```
### How was this patch tested?
New unit test suite, `AvroSerdeSuite`, was added to test corner cases on `AvroSerializer` and `AvroDeserializer` and verify that the exception messages are as expected. Existing tests in `AvroSuite` also continue to pass, with modifications in places where assertions were made about the exceptions that would be thrown.
Closes#31333 from xkrogen/xkrogen-SPARK-34182-avro-serde-errormessages.
Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Remove unused shuffleBlockResolver variable in SortShuffleWriter.
### Why are the changes needed?
For better code understanding.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
End to End.
Closes#31433 from offthewall123/remove_shuffleBlockResolver_in_SortShuffleWriter.
Authored-by: offthewall123 <dingyu.xu@intel.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR proposes to fix the UTs being added in SPARK-31793, so that all things contributing the length limit are properly accounted.
### Why are the changes needed?
The test `DataSourceScanExecRedactionSuite.SPARK-31793: FileSourceScanExec metadata should contain limited file paths` is failing conditionally, depending on the length of the temp directory.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Modified UTs explain the missing points, which also do the test.
Closes#31435 from HeartSaVioR/SPARK-34326.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
Replaces `collection.map(f1).flatten(f2)` with `collection.flatMap` if possible. it's semantically consistent, but looks simpler.
### Why are the changes needed?
Code Simpilefications.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#31416 from LuciferYang/SPARK-34310.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
This is an extension of PR #25613, I try to add a tip info for `Resource` column to make it easier for users to know what the column actually means and avoid confusion.
### Why are the changes needed?
After upgrading from 2.3.2 to 3.0.1, the new `Resources` column in the executors page is always blank because it does not use GPU/FPGA,
and there is no tip info, so users are often confused when they do not know the exact meaning of this column.
### Does this PR introduce _any_ user-facing change?
add a tip info in the executors page.
### How was this patch tested?
manual test works well as below:
![fixed](https://user-images.githubusercontent.com/52202080/106248350-d032ee00-624b-11eb-9ae4-92319ed11110.png)
Closes#31392 from akiyamaneko/executors-resources-tips.
Authored-by: neko <echohlne@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
### What changes were proposed in this pull request?
There are some deprecated API usage compilation warning related to Apache commons-io as follows:
```
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala:1109: [deprecation org.apache.spark.deploy.SparkSubmitSuite.checkDownloadedFile.$org_scalatest_assert_macro_expr.$org_scalatest_assert_macro_left | origin=org.apache.commons.io.FileUtils.readFileToString | version=] method readFileToString in class FileUtils is deprecated
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala:1110: [deprecation org.apache.spark.deploy.SparkSubmitSuite.checkDownloadedFile.$org_scalatest_assert_macro_expr.$org_scalatest_assert_macro_right | origin=org.apache.commons.io.FileUtils.readFileToString | version=] method readFileToString in class FileUtils is deprecated
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala:1152: [deprecation org.apache.spark.deploy.SparkSubmitSuite | origin=org.apache.commons.io.FileUtils.write | version=] method write in class FileUtils is deprecated
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala:1167: [deprecation org.apache.spark.deploy.SparkSubmitSuite | origin=org.apache.commons.io.FileUtils.write | version=] method write in class FileUtils is deprecated
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala:201: [deprecation org.apache.spark.deploy.history.HistoryServerSuite.<local HistoryServerSuite>.$anonfun.exp | origin=org.apache.commons.io.IOUtils.toString | version=] method toString in class IOUtils is deprecated
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala:716: [deprecation org.apache.spark.deploy.history.HistoryServerSuite.getContentAndCode.inString.$anonfun | origin=org.apache.commons.io.IOUtils.toString | version=] method toString in class IOUtils is deprecated
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala:732: [deprecation org.apache.spark.deploy.history.HistoryServerSuite.connectAndGetInputStream.errString.$anonfun | origin=org.apache.commons.io.IOUtils.toString | version=] method toString in class IOUtils is deprecated
[WARNING] [Warn] /spark/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala:267: [deprecation org.apache.spark.streaming.InputStreamsSuite.<local InputStreamsSuite>.$anonfun.$anonfun.write | origin=org.apache.commons.io.IOUtils.write | version=] method write in class IOUtils is deprecated
[WARNING] [Warn] /spark/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala:912: [deprecation org.apache.spark.streaming.StreamingContextSuite.createCorruptedCheckpoint | origin=org.apache.commons.io.FileUtils.write | version=] method write in class FileUtils is deprecated
```
The main API change is to need to add a `java.nio.charset.Charset` parameter when the corresponding method is called, so the main change of is pr is add a `StandardCharsets.UTF_8` parameter to the these method.
### Why are the changes needed?
Fix deprecated API usage of Apache commons-io.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#31389 from LuciferYang/SPARK-34284.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR aims to prevent `HeartbeatReceiver` asks `Executor` to re-register blocker manager when the SparkContext is already stopped.
### Why are the changes needed?
Currently, `HeartbeatReceiver` blindly asks re-registration for the new heartbeat message.
However, when SparkContext is stopped, we don't need to re-register new block manager.
Re-registration causes unnecessary executors' logs and and a delay on job termination.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the CIs with the newly added test case.
Closes#31373 from dongjoon-hyun/SPARK-34273.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR adds a new configuration, `spark.storage.blockManagerMasterDriverHeartbeatTimeoutMs`.
### Why are the changes needed?
Currently, it's a hard-coded `10 minutes`.
### Does this PR introduce _any_ user-facing change?
No. The default value is the same.
### How was this patch tested?
Pass the CIs.
Closes#31383 from dongjoon-hyun/SPARK-34278.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Use `count` to simplify `find + size(or length)` operation, it's semantically consistent, but looks simpler.
**Before**
```
seq.filter(p).size
```
**After**
```
seq.count(p)
```
### Why are the changes needed?
Code Simpilefications.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#31374 from LuciferYang/SPARK-34275.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Allow broadcast blocks to be put during decommissioning since migrations don't apply to them and they may be stored as part of job exec.
### Why are the changes needed?
Potential race condition.
### Does this PR introduce _any_ user-facing change?
Removal of race condition.
### How was this patch tested?
New unit test.
Closes#31298 from holdenk/SPARK-34193-torrentbroadcast-blockmanager-decommissioning-potential-race-condition.
Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Ensure that if a stage fails in the UI page, the corresponding error message can be displayed correctly.
### Why are the changes needed?
errormessage is not handled properly in JavaScript. If the 'at' is not exist, the error message on the page will be blank.
I made wochanges,
1. `msg.indexOf("at")` => `msg.indexOf("\n")`
![image](https://user-images.githubusercontent.com/52202080/105663531-7362cb00-5f0d-11eb-87fd-008ed65c33ca.png)
As shows ablove, truncated at the 'at' position will result in a strange abstract of the error message. If there is a `\n` worit is more reasonable to truncate at the '\n' position.
2. If the `\n` does not exist check whether the msg is more than 100. If true, then truncate the display to avoid too long error message
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test shows as belows, just a js change:
before modified:
![problem](https://user-images.githubusercontent.com/52202080/105712153-661cff00-5f54-11eb-80bf-e33c323c4e55.png)
after modified
![after mdified](https://user-images.githubusercontent.com/52202080/105712180-6c12e000-5f54-11eb-8998-ff8bc8a0a503.png)
Closes#31314 from akiyamaneko/error_message_display_empty.
Authored-by: neko <echohlne@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
1. Add back Maven enforcer for duplicate dependencies check
2. More strict check on Hadoop versions which support shaded client in `IsolatedClientLoader`. To do proper version check, this adds a util function `majorMinorPatchVersion` to extract major/minor/patch version from a string.
3. Cleanup unnecessary code
### Why are the changes needed?
The Maven enforcer was removed as part of #30556. This proposes to add it back.
Also, Hadoop shaded client doesn't work in certain cases (see [these comments](https://github.com/apache/spark/pull/30701#discussion_r558522227) for details). This strictly checks that the current Hadoop version (i.e., 3.2.2 at the moment) has good support of shaded client or otherwise fallback to old unshaded ones.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing tests.
Closes#31203 from sunchao/SPARK-33212-followup.
Lead-authored-by: Chao Sun <sunchao@apple.com>
Co-authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Using a function like `.mkString` or `.getLines` directly on a `scala.io.Source` opened by `fromFile`, `fromURL`, `fromURI ` will leak the underlying file handle, this pr use the `Utils.tryWithResource` method wrap the `BufferedSource` to ensure these `BufferedSource` closed.
### Why are the changes needed?
Avoid file handle leak.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#31323 from LuciferYang/source-not-closed.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Redact event SparkListenerEnvironmentUpdate in log when its processing time exceeded logSlowEventThreshold
### Why are the changes needed?
Credentials could be exposed when its processing time exceeded logSlowEventThreshold
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manually tested
Closes#31335 from warrenzhu25/34232.
Authored-by: Warren Zhu <warren.zhu25@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Compare the 3.1.1 API doc with the latest release version 3.0.1. Fix the following issues:
- Add missing `Since` annotation for new APIs
- Remove the leaking class/object in API doc
### Why are the changes needed?
Fix the issues in the Spark 3.1.1 release API docs.
### Does this PR introduce _any_ user-facing change?
Yes, API doc changes.
### How was this patch tested?
Manually test.
Closes#31271 from xuanyuanking/SPARK-34185.
Lead-authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Update Avro dependency to version 1.10.1
### Why are the changes needed?
To catch up multiple improvements of Avro as well as fix security issues on transitive dependencies.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Since there were no API changes required we just run the tests
Closes#31232 from iemejia/SPARK-27733-avro-upgrade.
Authored-by: Ismaël Mejía <iemejia@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Add executor number check.
### Why are the changes needed?
The test `decommission workers ensure that shuffle output is regenerated even with shuffle service` assumes it has two executor and both of two tasks can execute concurrently.
The two tasks will execute serially if there only one executor. The result is test is unexpceted. E.g.
```
[info] 5 did not equal 4 Expected 4 tasks but got List(0:0:0:0-SUCCESS, 0:0:1:0-FAILED, 0:0:1:1-SUCCESS, 0:1:0:0-SUCCESS, 1:0:0:0-SUCCESS) (DecommissionWorkerSuite.scala:190)
```
The failed task due to the first task finished and decommission the worker.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass exists test.
Closes#31255 from ulysses-you/SPARK-34166.
Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Check SPARK_TESTING as lazy val to avoid slow down when there are many environment variables
### Why are the changes needed?
If there are many environment variables, sys.env slows is very slow. As Utils.isTesting is called very often during Dataframe-Optimization, this can slow down evaluation very much.
An example for triggering the problem can be found in the bug ticket https://issues.apache.org/jira/browse/SPARK-34115
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
With the example provided in the ticket.
Closes#31244 from nob13/bug/34115.
Lead-authored-by: Norbert Schultz <norbert.schultz@reactivecore.de>
Co-authored-by: Norbert Schultz <noschultz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR aims to support fallback storage clean-up during stopping `SparkContext`.
### Why are the changes needed?
SPARK-33545 added `Support Fallback Storage during worker decommission` for the managed cloud-storages with TTL support. Usually, it's one day. This PR will add an additional clean-up feature during stopping `SparkContext` in order to save some money before TTL or the other HDFS-compatible storage which doesn't have TTL support.
### Does this PR introduce _any_ user-facing change?
Yes, but this is a new feature.
### How was this patch tested?
Pass the newly added UT.
Closes#31215 from dongjoon-hyun/SPARK-34142.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Display history server entries using datatables instead of Mustache + Datatables which proved to be faster and non-blocking for the webpage while searching (using search bar in the page)
### Why are the changes needed?
Small changes in the attempts (entries) and removed part of HTML (Mustache template).
### Does this PR introduce _any_ user-facing change?
Not very sure, but it's not supposed to change the way the page looks rather it changes how entries are displayed.
### How was this patch tested?
Running test, since it's not adding new functionality.
Closes#31191 from mohan3d/feat/history-server-ui-optimization.
Lead-authored-by: mohan3d <mohan3d94@gmail.com>
Co-authored-by: Author: mohan3d <mohan3d94@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
This:
1. switches Spark to use shaded Hadoop clients, namely hadoop-client-api and hadoop-client-runtime, for Hadoop 3.x.
2. upgrade built-in version for Hadoop 3.x to Hadoop 3.2.2
Note that for Hadoop 2.7, we'll still use the same modules such as hadoop-client.
In order to still keep default Hadoop profile to be hadoop-3.2, this defines the following Maven properties:
```
hadoop-client-api.artifact
hadoop-client-runtime.artifact
hadoop-client-minicluster.artifact
```
which default to:
```
hadoop-client-api
hadoop-client-runtime
hadoop-client-minicluster
```
but all switch to `hadoop-client` when the Hadoop profile is hadoop-2.7. A side affect from this is we'll import the same dependency multiple times. For this I have to disable Maven enforcer `banDuplicatePomDependencyVersions`.
Besides above, there are the following changes:
- explicitly add a few dependencies which are imported via transitive dependencies from Hadoop jars, but are removed from the shaded client jars.
- removed the use of `ProxyUriUtils.getPath` from `ApplicationMaster` which is a server-side/private API.
- modified `IsolatedClientLoader` to exclude `hadoop-auth` jars when Hadoop version is 3.x. This change should only matter when we're not sharing Hadoop classes with Spark (which is _mostly_ used in tests).
### Why are the changes needed?
Hadoop 3.2.2 is released with new features and bug fixes, so it's good for the Spark community to adopt it. However, latest Hadoop versions starting from Hadoop 3.2.1 have upgraded to use Guava 27+. In order to resolve Guava conflicts, this takes the approach by switching to shaded client jars provided by Hadoop. This also has the benefits of avoid pulling other 3rd party dependencies from Hadoop side so as to avoid more potential future conflicts.
### Does this PR introduce _any_ user-facing change?
When people use Spark with `hadoop-provided` option, they should make sure class path contains `hadoop-client-api` and `hadoop-client-runtime` jars. In addition, they may need to make sure these jars appear before other Hadoop jars in the order. Otherwise, classes may be loaded from the other non-shaded Hadoop jars and cause potential conflicts.
### How was this patch tested?
Relying on existing tests.
Closes#30701 from sunchao/test-hadoop-3.2.2.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Some local variables are declared as `var`, but they are never reassigned and should be declared as `val`, so this pr turn these from `var` to `val` except for `mockito` related cases.
### Why are the changes needed?
Use `val` instead of `var` when possible.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#31142 from LuciferYang/SPARK-33346.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
This pr use `exists` or `forall` to simplify `filter + emptiness check`, it's semantically consistent, but looks simpler. The rule as follow:
- `seq.filter(p).size == 0)` -> `!seq.exists(p)`
- `seq.filter(p).length > 0` -> `seq.exists(p)`
- `seq.filterNot(p).isEmpty` -> `seq.forall(p)`
- `seq.filterNot(p).nonEmpty` -> `!seq.forall(p)`
### Why are the changes needed?
Code Simpilefications.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#31184 from LuciferYang/SPARK-34118.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Fix a regression from https://github.com/apache/spark/pull/29959.
In Spark, the following file paths are considered as hidden paths and they are ignored on file reads:
1. starts with "_" and doesn't contain "="
2. starts with "."
However, after the refactoring PR https://github.com/apache/spark/pull/29959, the hidden paths are not filtered out on partition inference: https://github.com/apache/spark/pull/29959/files#r556432426
This PR is to fix the bug. To archive the goal, the method `InMemoryFileIndex.shouldFilterOut` is refactored as `HadoopFSUtils.shouldFilterOutPathName`
### Why are the changes needed?
Bugfix
### Does this PR introduce _any_ user-facing change?
Yes, it fixes a bug for reading file paths with partitions.
### How was this patch tested?
Unit test
Closes#31169 from gengliangwang/fileListingBug.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
There are some redundant collection conversion can be removed, for version compatibility, clean up these with Scala-2.13 profile.
### Why are the changes needed?
Remove redundant collection conversion
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
- Pass the Jenkins or GitHub Action
- Manual test `core`, `graphx`, `mllib`, `mllib-local`, `sql`, `yarn`,`kafka-0-10` in Scala 2.13 passed
Closes#31125 from LuciferYang/SPARK-34068.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
This pr use `exists` to simplify `find + emptiness check`, it's semantically consistent, but looks simpler.
**Before**
```
seq.find(p).isDefined
or
seq.find(p).isEmpty
```
**After**
```
seq.exists(p)
or
!seq.exists(p)
```
### Why are the changes needed?
Code Simpilefications.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#31130 from LuciferYang/SPARK-34070.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Add min threshold time speculation config
### Why are the changes needed?
When we turn on speculation with default configs we have the last 10% of the tasks subject to speculation. There are a lot of stages where the stage runs for few seconds to minutes. Also in general we don't want to speculate tasks that run within a minimum threshold. By setting a minimum threshold for speculation config gives us better control for speculative tasks
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit test
Closes#30710 from redsanket/SPARK-33741.
Lead-authored-by: schintap <schintap@verizonmedia.com>
Co-authored-by: Sanket Chintapalli <chintapalli.sanketreddy@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
### What changes were proposed in this pull request?
Increase test timeout.
### Why are the changes needed?
It's more reasonable to use 60s instead of 6s since many code place use the 60s in `TestUtils.waitUntilExecutorsUp`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass exists test.
Closes#31166 from ulysses-you/SPARK-32850-FOLLOWUP.
Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Add shouldInterruptTaskThread check when kill barrier task.
### Why are the changes needed?
We should interrupt task thread if user set local property `SPARK_JOB_INTERRUPT_ON_CANCEL` to true.
### Does this PR introduce _any_ user-facing change?
Yes, task will be interrupted if user set `SPARK_JOB_INTERRUPT_ON_CANCEL` to true.
### How was this patch tested?
Add test.
Closes#31127 from ulysses-you/SPARK-34069.
Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
### What changes were proposed in this pull request?
Missing POD detection is extended by timestamp (and time limit) based check to avoid wrongfully detection of missing POD detection.
The two new timestamps:
- `fullSnapshotTs` is introduced for the `ExecutorPodsSnapshot` which only updated by the pod polling snapshot source
- `registrationTs` is introduced for the `ExecutorData` and it is initialized at the executor registration at the scheduler backend
Moreover a new config `spark.kubernetes.executor.missingPodDetectDelta` is used to specify the accepted delta between the two.
### Why are the changes needed?
Watching a POD (`ExecutorPodsWatchSnapshotSource`) only inform about single POD changes. This could wrongfully lead to detecting of missing PODs (PODs known by scheduler backend but missing from POD snapshots) by the executor POD lifecycle manager.
A key indicator of this error is seeing this log message:
> "The executor with ID [some_id] was not found in the cluster but we didn't get a reason why. Marking the executor as failed. The executor may have been deleted but the driver missed the deletion event."
So one of the problem is running the missing POD detection check even when a single POD is changed without having a full consistent snapshot about all the PODs (see `ExecutorPodsPollingSnapshotSource`).
The other problem could be the race between the executor POD lifecycle manager and the scheduler backend: so even in case of a having a full snapshot the registration at the scheduler backend could precede the snapshot polling (and processing of those polled snapshots).
### Does this PR introduce _any_ user-facing change?
Yes. When the POD is missing then the reason message explaining the executor's exit is extended with both timestamps (the polling time and the executor registration time) and even the new config is mentioned.
### How was this patch tested?
The existing unit tests are extended.
Closes#30675 from attilapiros/SPARK-33711.
Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
### What changes were proposed in this pull request?
For `AllJobsPage `class, `AllJobsPage` gets the schedulingMode of enumerated type by loading the `spark.scheduler.mode `configuration from Sparkconf, but an enumeration conversion error occurs when I set the value of this configuration to lowercase.
The reason for this problem is that the value of the `SchedulingMode `enumeration class is uppercase, which occurs when I configure `spark.scheduler.mode` to be lowercase.
I saw that the `#org.apache.spark.scheduler.TaskSchedulerImpl` class convert the `spark. scheduler.mode` value to uppercase, so I think it should be converted in `AllJobsPage `as well.
### Why are the changes needed?
An enumerated conversion error occurred with Spark when I set the value of this configuration to lowercase.
### How was this patch tested?
Existing tests.
Closes#31015 from yikf/master.
Authored-by: yikf <13468507104@163.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
According to https://github.com/apache/spark/pull/29966#discussion_r554514344
Use wrong name about suite file, this pr to fix this problem.
And change to use some fake ivy link for this test
### Why are the changes needed?
Follow file name rule
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
No
Closes#31118 from AngersZhuuuu/SPARK-33084-FOLLOW-UP.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR takes over https://github.com/apache/spark/pull/28780.
1. Counted the spilled memory size when creating the `UnsafeExternalSorter` with the existing `InMemorySorter`
2. Accumulate the `totalSpillBytes` when merging two `UnsafeExternalSorter`
### Why are the changes needed?
As mentioned in https://github.com/apache/spark/pull/28780:
> It happends when hash aggregate downgrades to sort based aggregate.
`UnsafeExternalSorter.createWithExistingInMemorySorter` calls spill on an `InMemorySorter` immediately, but the memory pointed by `InMemorySorter` is acquired by outside `BytesToBytesMap`, instead the allocatedPages in `UnsafeExternalSorter`. So the memory spill bytes metric is always 0, but disk bytes spill metric is right.
Besides, this PR also fixes the `UnsafeExternalSorter.merge` by accumulating the `totalSpillBytes` of two sorters. Thus, we can report the correct spilled size in `HashAggregateExec.finishAggregate`.
Issues can be reproduced by the following step by checking the SQL metrics in UI:
```
bin/spark-shell --driver-memory 512m --executor-memory 512m --executor-cores 1 --conf "spark.default.parallelism=1"
scala> sql("select id, count(1) from range(10000000) group by id").write.csv("/tmp/result.json")
```
Before:
<img width="200" alt="WeChatfe5146180d91015e03b9a27852e9a443" src="https://user-images.githubusercontent.com/16397174/103625414-e6fc6280-4f75-11eb-8b93-c55095bdb5b8.png">
After:
<img width="200" alt="WeChat42ab0e73c5fbc3b14c12ab85d232071d" src="https://user-images.githubusercontent.com/16397174/103625420-e8c62600-4f75-11eb-8e1f-6f5e8ab561b9.png">
### Does this PR introduce _any_ user-facing change?
Yes, users can see the correct spill metrics after this PR.
### How was this patch tested?
Tested manually and added UTs.
Closes#31035 from Ngone51/SPARK-31952.
Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: wangguangxin.cn <wangguangxin.cn@bytedance.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR is basically a followup of https://github.com/apache/spark/pull/14332.
Calling `map` alone might leave it not executed due to lazy evaluation, e.g.)
```
scala> val foo = Seq(1,2,3)
foo: Seq[Int] = List(1, 2, 3)
scala> foo.map(println)
1
2
3
res0: Seq[Unit] = List((), (), ())
scala> foo.view.map(println)
res1: scala.collection.SeqView[Unit,Seq[_]] = SeqViewM(...)
scala> foo.view.foreach(println)
1
2
3
```
We should better use `foreach` to make sure it's executed where the output is unused or `Unit`.
### Why are the changes needed?
To prevent the potential issues by not executing `map`.
### Does this PR introduce _any_ user-facing change?
No, the current codes look not causing any problem for now.
### How was this patch tested?
I found these item by running IntelliJ inspection, double checked one by one, and fixed them. These should be all instances across the codebase ideally.
Closes#31110 from HyukjinKwon/SPARK-34059.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
### What changes were proposed in this pull request?
This is the shuffle writer side change where executors can push data to remote shuffle services. This is needed for push-based shuffle - SPIP [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602).
Summary of changes:
- This adds support for executors to push shuffle blocks after map tasks complete writing shuffle data.
- This also introduces a timeout specifically for creating connection to remote shuffle services.
### 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).
- The main reason to create a separate connection creation timeout is because the existing `connectionTimeoutMs` is overloaded and is used for connection creation timeouts as well as connection idle timeout. The connection creation timeout should be much lower than the idle timeouts. The default for `connectionTimeoutMs` is 120s. This is quite high for just establishing the connections. If a shuffle server node is bad then the connection creation will fail within few seconds. However, an overloaded shuffle server may take much longer to respond to a request and the channel can stay idle for a much longer time which is expected. Another reason is that with push-based shuffle, an executor may be fetching shuffle data and pushing shuffle data (next stage) simultaneously. Both these tasks will share the same connections with the shuffle service. If there is a bad shuffle server node and the connection creation timeout is very high then both these tasks end up waiting a long time time eventually impacting the performance.
### Does this PR introduce _any_ user-facing change?
Yes. This PR introduces client-side configs for push-based shuffle. If push-based shuffle is turned-off then the users will not see any change.
### 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: Min Shen mshenlinkedin.com
Co-authored-by: Chandni Singh chsinghlinkedin.com
Co-authored-by: Ye Zhou yezhoulinkedin.com
Closes#30312 from otterc/SPARK-32917.
Lead-authored-by: Chandni Singh <singh.chandni@gmail.com>
Co-authored-by: Chandni Singh <chsingh@linkedin.com>
Co-authored-by: Min Shen <mshen@linked.in.com>
Co-authored-by: Ye Zhou <yezhou@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
### What changes were proposed in this pull request?
This PR makes `AppStatusListener` update the peak memory metrics for each Executor on task end like other peak memory metrics (e.g, stage, executors in a stage).
### Why are the changes needed?
When `AppStatusListener#onExecutorMetricsUpdate` is called, peak memory metrics for Executors, stages and executors in a stage are updated but currently, the metrics only for Executors are not updated on task end.
### Does this PR introduce _any_ user-facing change?
Yes. Executor peak memory metrics is updated more accurately.
### How was this patch tested?
After I run a job with `local-cluster[1,1,1024]` and visited `/api/v1/<appid>/executors`, I confirmed `peakExecutorMemory` metrics is shown for an Executor even though the life time of each job is very short .
I also modify the json files for `HistoryServerSuite`.
Closes#31029 from sarutak/update-executor-metrics-on-taskend.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Filter out the driver entity when updating the exclusion status of live executors(including the driver), so the UI won't be marked as excluded in the UI even if the node that hosts the driver has been marked as excluded.
### Why are the changes needed?
Before this change, if we run spark with the standalone mode and with spark.blacklist.enabled=true. The driver will be marked as excluded when the host that hosts that driver has been marked as excluded. While it's incorrect because the exclude list feature will exclude executors only and the driver is still active.
![image](https://user-images.githubusercontent.com/26694233/103238740-35c05180-4911-11eb-99a2-c87c059ba0cf.png)
After the fix, the driver won't be marked as excluded.
![image](https://user-images.githubusercontent.com/26694233/103238806-6f915800-4911-11eb-80d5-3c99266cfd0a.png)
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test. Reopen the UI and see the driver is no longer marked as excluded.
Closes#30954 from baohe-zhang/SPARK-33029.
Authored-by: Baohe Zhang <baohe.zhang@verizonmedia.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
From below log, Stage 600 could be removed from `stageAttemptToNumSpeculativeTasks` by `onStageCompleted()`, but the speculative task 306.1 in stage 600 threw `NoSuchElementException` when it entered into `onTaskEnd()`.
```
21/01/04 03:00:32,259 WARN [task-result-getter-2] scheduler.TaskSetManager:69 : Lost task 306.1 in stage 600.0 (TID 283610, hdc49-mcc10-01-0510-4108-039-tess0097.stratus.rno.ebay.com, executor 27): TaskKilled (another attempt succeeded)
21/01/04 03:00:32,259 INFO [task-result-getter-2] scheduler.TaskSetManager:57 : Task 306.1 in stage 600.0 (TID 283610) failed, but the task will not be re-executed (either because the task failed with a shuffle data fetch failure, so the
previous stage needs to be re-run, or because a different copy of the task has already succeeded).
21/01/04 03:00:32,259 INFO [task-result-getter-2] cluster.YarnClusterScheduler:57 : Removed TaskSet 600.0, whose tasks have all completed, from pool default
21/01/04 03:00:32,259 INFO [HiveServer2-Handler-Pool: Thread-5853] thriftserver.SparkExecuteStatementOperation:190 : Returning result set with 50 rows from offsets [5378600, 5378650) with 1fe245f8-a7f9-4ec0-bcb5-8cf324cbbb47
21/01/04 03:00:32,260 ERROR [spark-listener-group-executorManagement] scheduler.AsyncEventQueue:94 : Listener ExecutorAllocationListener threw an exception
java.util.NoSuchElementException: key not found: Stage 600 (Attempt 0)
at scala.collection.MapLike.default(MapLike.scala:235)
at scala.collection.MapLike.default$(MapLike.scala:234)
at scala.collection.AbstractMap.default(Map.scala:63)
at scala.collection.mutable.HashMap.apply(HashMap.scala:69)
at org.apache.spark.ExecutorAllocationManager$ExecutorAllocationListener.onTaskEnd(ExecutorAllocationManager.scala:621)
at org.apache.spark.scheduler.SparkListenerBus.doPostEvent(SparkListenerBus.scala:45)
at org.apache.spark.scheduler.SparkListenerBus.doPostEvent$(SparkListenerBus.scala:28)
at org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:38)
at org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:38)
at org.apache.spark.util.ListenerBus.postToAll(ListenerBus.scala:115)
at org.apache.spark.util.ListenerBus.postToAll$(ListenerBus.scala:99)
at org.apache.spark.scheduler.AsyncEventQueue.super$postToAll(AsyncEventQueue.scala:116)
at org.apache.spark.scheduler.AsyncEventQueue.$anonfun$dispatch$1(AsyncEventQueue.scala:116)
at scala.util.DynamicVariable.withValue(DynamicVariable.scala:62)
at org.apache.spark.scheduler.AsyncEventQueue.org$apache$spark$scheduler$AsyncEventQueue$$dispatch(AsyncEventQueue.scala:102)
at org.apache.spark.scheduler.AsyncEventQueue$$anon$2.$anonfun$run$1(AsyncEventQueue.scala:97)
at org.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1320)
at org.apache.spark.scheduler.AsyncEventQueue$$anon$2.run(AsyncEventQueue.scala:97)
```
### Why are the changes needed?
To avoid throwing the java.util.NoSuchElementException
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
This is a protective patch and it's not easy to reproduce in UT due to the event order is not fixed in a async queue.
Closes#31025 from LantaoJin/SPARK-34000.
Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Fix un-correct doc of last change https://github.com/apache/spark/pull/30922#discussion_r551453193
### Why are the changes needed?
FIx doc
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Builds finished correctly.
Closes#31016 from AngersZhuuuu/SPARK-33908-FOLLOW-UP.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR upgrade Py4J from 0.10.9 to 0.10.9.1 that contains some bug fixes and improvements.
It contains one bug fix (4152353ac1).
### Why are the changes needed?
To leverage fixes from the upstream in Py4J.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Jenkins build and GitHub Actions will test it out.
Closes#31009 from HyukjinKwon/SPARK-33984.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Check if the executorSummary.peakMemoryMetrics is defined before accessing it. Without checking, the UI has risked being stuck at the Executors page.
### Why are the changes needed?
App live UI may stuck at Executors page without this fix.
Steps to reproduce (with master branch):
In mac OS standalone mode, open a spark-shell
$SPARK_HOME/bin/spark-shell --master spark://localhost:7077
val x = sc.makeRDD(1 to 100000, 5)
x.count()
Then open the app UI in the browser, and click the Executors page, will get stuck at this page:
![image](https://user-images.githubusercontent.com/26694233/103105677-ca1a7380-45f4-11eb-9245-c69f4a4e816b.png)
Also, the return JSON from API endpoint http://localhost:4040/api/v1/applications/app-20201224134418-0003/executors miss "peakMemoryMetrics" for executor objects. I attached the full json text in https://issues.apache.org/jira/browse/SPARK-33906.
I debugged it and observed that ExecutorMetricsPoller
.getExecutorUpdates returns an empty map, which causes peakExecutorMetrics to None in https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/status/LiveEntity.scala#L345. The possible reason for returning the empty map is that the stage completion time is shorter than the heartbeat interval, so the stage entry in stageTCMP has already been removed before the reportHeartbeat is called.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test, rerun the steps of bug reproduce and see the bug is gone.
Closes#30920 from baohe-zhang/SPARK-33906.
Authored-by: Baohe Zhang <baohe.zhang@verizonmedia.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR proposed to make worker/executor decommission signal configurable.
* Added confs: `spark.worker.decommission.signal` / `spark.executor.decommission.signal`
* Rename `WorkerSigPWRReceived`/ `ExecutorSigPWRReceived` to `WorkerDecomSigReceived`/ `ExecutorDecomSigReceived`
### Why are the changes needed?
The current signal `PWR` can't work on macOS since it's not compliant with POSIX while macOS does. So the developers currently can't do end-to-end decommission test on their macOS environment.
Besides, the configuration becomes more flexible for users in case the default signal (`PWR`) gets conflicted with their own applications/environment.
### Does this PR introduce _any_ user-facing change?
No (it's a new API for 3.2)
### How was this patch tested?
Manually tested.
Closes#30968 from Ngone51/configurable-decom-signal.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
There are only 3 compilation warnings related to `view bounds are deprecated` in Spark Code:
```
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala:35: view bounds are deprecated; use an implicit parameter instead.
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala:35: view bounds are deprecated; use an implicit parameter instead.
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala:55: view bounds are deprecated; use an implicit parameter instead.
```
This pr try to fix these compilation warnings.
### Why are the changes needed?
Fix compilation warnings about ` view bounds are deprecated`
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#30924 from LuciferYang/SPARK-33804.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Use the testing mode for the test to fix the flaky.
### Why are the changes needed?
The test is flaky:
```scala
[info] - SPARK-23365 Don't update target num executors when killing idle executors *** FAILED *** (126 milliseconds)
[info] 1 did not equal 2 (ExecutorAllocationManagerSuite.scala:1615)
[info] org.scalatest.exceptions.TestFailedException:
[info] at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:530)
[info] at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:529)
[info] at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
[info] at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:503)
[info] at org.apache.spark.ExecutorAllocationManagerSuite.$anonfun$new$84(ExecutorAllocationManagerSuite.scala:1617)
...
```
The root cause should be the same as https://github.com/apache/spark/pull/29773 since the test run under non-testing mode.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manually checked. Flaky is gone by running the test hundreds of times after this fix.
Closes#30956 from Ngone51/fix-flaky-SPARK-23365.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This is kind of a followup of https://github.com/apache/spark/pull/24033.
The first and last usage of that argument `SecurityManager` was removed in https://github.com/apache/spark/pull/24033.
After that, we don't need to pass `SecurityManager` anymore in `Utils.fetchFile` and related code paths.
This PR proposes to remove it out.
### Why are the changes needed?
For better readability of codes.
### Does this PR introduce _any_ user-facing change?
No, dev-only.
### How was this patch tested?
Manually complied. GitHub Actions and Jenkins build should test it out as well.
Closes#30945 from HyukjinKwon/SPARK-33925.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR aims to fix offset bug and improve compression codec test coverage.
### Why are the changes needed?
When the user choose a non-default codec, it causes a failure.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the extended test suite.
Closes#30934 from dongjoon-hyun/SPARK-33916.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Per discuss in https://github.com/apache/spark/pull/29966#discussion_r531917374
We'd better change `SparkSubmitUtils.resolveMavenCoordinates()` 's return value as `Seq[String]`
### Why are the changes needed?
refactor code
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Existed UT
Closes#30922 from AngersZhuuuu/SPARK-33908.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
There are total 15 compilation warnings about `Unicode escapes in triple quoted strings are deprecated` in Spark code now:
```
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2930: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2931: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2932: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2933: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2934: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2935: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2936: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2937: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala:82: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala:32: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala:79: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala:97: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala:101: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala:76: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala:83: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
```
This pr try to fix these warnnings.
### Why are the changes needed?
Cleanup compilation warnings about `Unicode escapes in triple quoted strings are deprecated`
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#30926 from LuciferYang/SPARK-33801.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Support add jar with ivy path
### Why are the changes needed?
Since submit app can support ivy, add jar we can also support ivy now.
### Does this PR introduce _any_ user-facing change?
User can add jar with sql like
```
add jar ivy:://group:artifict:version?exclude=xxx,xxx&transitive=true
add jar ivy:://group:artifict:version?exclude=xxx,xxx&transitive=false
```
core api
```
sparkContext.addJar("ivy:://group:artifict:version?exclude=xxx,xxx&transitive=true")
sparkContext.addJar("ivy:://group:artifict:version?exclude=xxx,xxx&transitive=false")
```
#### Doc Update snapshot
![image](https://user-images.githubusercontent.com/46485123/101227738-de451200-36d3-11eb-813d-78a8b879da4f.png)
### How was this patch tested?
Added UT
Closes#29966 from AngersZhuuuu/support-add-jar-ivy.
Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
This PR aims to exclude fallback block manager from `executorList` function.
### Why are the changes needed?
When a fallback storage is used, the executors UI tab hangs because the executor list REST API result doesn't have `peakMemoryMetrics` of `ExecutorMetrics`. The root cause is that the block manager id used by fallback storage is included in the API result and it doesn't have `peakMemoryMetrics` because it's populated during HeartBeat reporting. We should hide it.
### Does this PR introduce _any_ user-facing change?
No. This is a bug fix on UI.
### How was this patch tested?
Manual. Run the following and visit Spark `executors` tab UI with browser.
```
bin/spark-shell -c spark.storage.decommission.fallbackStorage.path=file:///tmp/spark-storage/
```
Closes#30911 from dongjoon-hyun/SPARK-33893.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This is a retry of #30177.
This is not a complete fix, but it would take long time to complete (#30242).
As discussed offline, at least using `ContextAwareIterator` should be helpful enough for many cases.
As the Python evaluation consumes the parent iterator in a separate thread, it could consume more data from the parent even after the task ends and the parent is closed. Thus, we should use `ContextAwareIterator` to stop consuming after the task ends.
### Why are the changes needed?
Python/Pandas UDF right after off-heap vectorized reader could cause executor crash.
E.g.,:
```py
spark.range(0, 100000, 1, 1).write.parquet(path)
spark.conf.set("spark.sql.columnVector.offheap.enabled", True)
def f(x):
return 0
fUdf = udf(f, LongType())
spark.read.parquet(path).select(fUdf('id')).head()
```
This is because, the Python evaluation consumes the parent iterator in a separate thread and it consumes more data from the parent even after the task ends and the parent is closed. If an off-heap column vector exists in the parent iterator, it could cause segmentation fault which crashes the executor.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added tests, and manually.
Closes#30899 from ueshin/issues/SPARK-33277/context_aware_iterator.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR aims to enable `spark.storage.replication.proactive` by default for Apache Spark 3.2.0.
### Why are the changes needed?
`spark.storage.replication.proactive` is added by SPARK-15355 at Apache Spark 2.2.0 and has been helpful when the block manager loss occurs frequently like K8s environment.
### Does this PR introduce _any_ user-facing change?
Yes, this will make the Spark jobs more robust.
### How was this patch tested?
Pass the existing UTs.
Closes#30876 from dongjoon-hyun/SPARK-33870.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR removed an unused variable `CompressionCodec.DEFAULT_COMPRESSION_CODEC`.
### Why are the changes needed?
Apache Spark 3.0.0 centralized this default value to `IO_COMPRESSION_CODEC.defaultValue` via [SPARK-26462](https://github.com/apache/spark/pull/23447).
We had better remove this variable to avoid any potential confusion in the future.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the CI compilation.
Closes#30880 from dongjoon-hyun/minor.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR aims to test all compression codecs for encrypted spilling.
### Why are the changes needed?
To improve test coverage. Currently, only `CompressionCodec.DEFAULT_COMPRESSION_CODEC` is under testing.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the CIs with the updated test cases.
Closes#30879 from dongjoon-hyun/SPARK-33873.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This is a followup PR for #30573 .
After this change applied, stage memory metrics will be updated on stage end.
### Why are the changes needed?
After #30573, executor memory metrics is updated on stage end but stage memory metrics is not updated.
It's better to update both metrics like `updateStageLevelPeakExecutorMetrics` does.
### Does this PR introduce _any_ user-facing change?
Yes. stage memory metrics is updated more accurately.
### How was this patch tested?
After I run a job and visited `/api/v1/<appid>/stages`, I confirmed `peakExecutorMemory` metrics is shown even though the life time of each stage is very short .
I also modify the json files for `HistoryServerSuite`.
Closes#30858 from sarutak/followup-SPARK-26341.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
Make MapIterator of BytesToBytesMap `hasNext` method idempotent
### Why are the changes needed?
The `hasNext` maybe called multiple times, if not guarded, second call of hasNext method after reaching the end of iterator will throw NoSuchElement exception.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
Update a unit test to cover this case.
Closes#30728 from advancedxy/SPARK-33756.
Authored-by: Xianjin YE <advancedxy@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Use `local[2]` to let tasks launch at the same time. And change counters (`numOnTaskXXX`) to `AtomicInteger` type to ensure thread safe.
### Why are the changes needed?
The test is still flaky after the fix https://github.com/apache/spark/pull/30072. See: https://github.com/apache/spark/pull/30728/checks?check_run_id=1557987642
And it's easy to reproduce if you test it multiple times (e.g. 100) locally.
The test sets up a stage with 2 tasks to run on an executor with 1 core. So these 2 tasks have to be launched one by one.
The task-2 will be launched after task-1 fails. However, since we don't retry failed task in local mode (MAX_LOCAL_TASK_FAILURES = 1), the stage will abort right away after task-1 fail and cancels the running task-2 at the same time. There's a chance that task-2 gets canceled before calling `PluginContainer.onTaskStart`, which leads to the test failure.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Tested manually after the fix and the test is no longer flaky.
Closes#30823 from Ngone51/debug-flaky-spark-33088.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
When driver stopping, pending rpc requests will cause error like:
> 17/12/12 18:30:16 ERROR TransportRequestHandler: Error while invoking RpcHandler#receive() for one-way message.
org.apache.spark.SparkException: Could not find CoarseGrainedScheduler.
at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:154)
at org.apache.spark.rpc.netty.Dispatcher.postOneWayMessage(Dispatcher.scala:134)
at org.apache.spark.rpc.netty.NettyRpcHandler.receive(NettyRpcEnv.scala:570)
at org.apache.spark.network.server.TransportRequestHandler.processOneWayMessage(TransportRequestHandler.java:180)
at org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:109)
at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:119)
at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
Or like:
> 17/12/12 18:20:44 INFO MemoryStore: MemoryStore cleared
17/12/12 18:20:44 INFO BlockManager: BlockManager stopped
17/12/12 18:20:44 INFO BlockManagerMaster: BlockManagerMaster stopped
17/12/12 18:20:44 ERROR TransportRequestHandler: Error while invoking RpcHandler#receive() for one-way message.
org.apache.spark.rpc.RpcEnvStoppedException: RpcEnv already stopped.
at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:152)
at org.apache.spark.rpc.netty.Dispatcher.postOneWayMessage(Dispatcher.scala:134)
at org.apache.spark.rpc.netty.NettyRpcHandler.receive(NettyRpcEnv.scala:570)
These are because CoarseGrainedScheduler and rpcEnv are already stopped, they're not error.
The related issue SPARK-22769 was opened on 2017, but the author didn't finish the pull request, so reopen this issue.
### How was this patch tested?
Existing tests
Closes#30658 from sqlwindspeaker/donot-log-rpc-error.
Authored-by: suqilong <suqilong@qiyi.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Initiate the `masterWebUiUrl` with the `webUi. webUrl` instead of the `masterPublicAddress`.
### Why are the changes needed?
Since [SPARK-21642](https://issues.apache.org/jira/browse/SPARK-21642), `WebUI` has changed from `localHostName` to `localCanonicalHostName` as the hostname to set up the web UI. However, the `masterPublicAddress` is from `RpcEnv`'s host address, which still uses `localHostName`. As a result, it returns the wrong Master web URL to the Worker.
### Does this PR introduce _any_ user-facing change?
Yes, when users click "Back to Master" in the Worker page:
Before this PR:
<img width="3258" alt="WeChat4acbfd163f51c76a5f9bc388c7479785" src="https://user-images.githubusercontent.com/16397174/102057951-b9664280-3e29-11eb-8749-5ee293902bdf.png">
After this PR:
![image](https://user-images.githubusercontent.com/16397174/102058016-d438b700-3e29-11eb-8641-a23a6b2f542e.png)
(Return to the Master page successfully.)
### How was this patch tested?
Tested manually.
Closes#30759 from Ngone51/fix-back-to-master.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
This PR aims to convert `EventLogFileReader`'s derived classes into `package private`.
- SingleFileEventLogFileReader
- RollingEventLogFilesFileReader
`EventLogFileReader` itself is used in `scheduler` module during tests.
### Why are the changes needed?
This classes were designed to be internal. This PR hides it explicitly to reduce the maintenance burden.
### Does this PR introduce _any_ user-facing change?
Yes, but these were exposed accidentally.
### How was this patch tested?
Pass CIs.
Closes#30814 from dongjoon-hyun/SPARK-33790.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
`FsHistoryProvider#checkForLogs` already has `FileStatus` when constructing `SingleFileEventLogFileReader`, and there is no need to get the `FileStatus` again when `SingleFileEventLogFileReader#fileSizeForLastIndex`.
### Why are the changes needed?
This can reduce a lot of rpc calls and improve the speed of the history server.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
exist ut
Closes#30780 from cxzl25/SPARK-33790.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
This PR introduces a helper method `withExecutor` that handles the creation of an Executor object and ensures that it is always stopped in a finally block. The tests in ExecutorSuite have been refactored to use this method.
### Why are the changes needed?
Recently an issue was discovered that leaked Executors (which are not explicitly stopped after a test) can cause other tests to fail due to the JVM being killed after 10 min. It is therefore crucial that tests always stop the Executor. By introducing this helper method, a simple pattern is established that can be easily adopted in new tests, which reduces the risk of regressions.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Run the ExecutorSuite locally.
Closes#30783 from sander-goos/SPARK-33793-close-executors.
Authored-by: Sander Goos <sander.goos@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR proposes:
- Respect `PYSPARK_PYTHON` and `PYSPARK_DRIVER_PYTHON` environment variables, or `spark.pyspark.python` and `spark.pyspark.driver.python` configurations in Kubernates just like other cluster types in Spark.
- Depreate `spark.kubernetes.pyspark.pythonVersion` and guide users to set the environment variables and configurations for Python executables.
NOTE that `spark.kubernetes.pyspark.pythonVersion` is already a no-op configuration without this PR. Default is `3` and other values are disallowed.
- In order for Python executable settings to be consistently used, fix `spark.archives` option to unpack into the current working directory in the driver of Kubernates' cluster mode. This behaviour is identical with Yarn's cluster mode. By doing this, users can leverage Conda or virtuenenv in cluster mode as below:
```python
conda create -y -n pyspark_conda_env -c conda-forge pyarrow pandas conda-pack
conda activate pyspark_conda_env
conda pack -f -o pyspark_conda_env.tar.gz
PYSPARK_PYTHON=./environment/bin/python spark-submit --archives pyspark_conda_env.tar.gz#environment app.py
```
- Removed several unused or useless codes such as `extractS3Key` and `renameResourcesToLocalFS`
### Why are the changes needed?
- To provide a consistent support of PySpark by using `PYSPARK_PYTHON` and `PYSPARK_DRIVER_PYTHON` environment variables, or `spark.pyspark.python` and `spark.pyspark.driver.python` configurations.
- To provide Conda and virtualenv support via `spark.archives` options.
### Does this PR introduce _any_ user-facing change?
Yes:
- `spark.kubernetes.pyspark.pythonVersion` is deprecated.
- `PYSPARK_PYTHON` and `PYSPARK_DRIVER_PYTHON` environment variables, and `spark.pyspark.python` and `spark.pyspark.driver.python` configurations are respected.
### How was this patch tested?
Manually tested via:
```bash
minikube delete
minikube start --cpus 12 --memory 16384
kubectl create namespace spark-integration-test
cat <<EOF | kubectl apply -f -
apiVersion: v1
kind: ServiceAccount
metadata:
name: spark
namespace: spark-integration-test
EOF
kubectl create clusterrolebinding spark-role --clusterrole=edit --serviceaccount=spark-integration-test:spark --namespace=spark-integration-test
dev/make-distribution.sh --pip --tgz -Pkubernetes
resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh --spark-tgz `pwd`/spark-3.2.0-SNAPSHOT-bin-3.2.0.tgz --service-account spark --namespace spark-integration-test
```
Unittests were also added.
Closes#30735 from HyukjinKwon/SPARK-33748.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
TO FIX flaky tests:
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132345/testReport/
```
org.apache.spark.sql.hive.thriftserver.HiveThriftHttpServerSuite.JDBC query execution
org.apache.spark.sql.hive.thriftserver.HiveThriftHttpServerSuite.Checks Hive version
org.apache.spark.sql.hive.thriftserver.HiveThriftHttpServerSuite.SPARK-24829 Checks cast as float
```
The root cause here is a jar conflict issue.
`NewCookie.isHttpOnly` is not defined in the `jsr311-api.jar` which conflicts
The transitive artifact `jsr311-api.jar` of `hadoop-client` is excluded at the maven side. See https://issues.apache.org/jira/browse/SPARK-27179.
The Jenkins PR builder and Github Action use `SBT` as the compiler tool.
First, the exclusion rule from maven is not followed by sbt, so I was able to see `jsr311-api.jar` from maven cache to be added to the classpath directly. **This seems to be a bug of `sbt-pom-reader` plugin but I'm not that sure.**
Then I added an `ExcludeRule` for the `hive-thriftserver` module at the SBT side and did see the `jsr311-api.jar` gone, but the CI jobs still failed with the same error.
I added a trace log in ThriftHttpServlet
```s
ERROR ThriftHttpServlet: !!!!!!!!! Suspect???????? --->
file:/home/jenkins/workspace/SparkPullRequestBuilder/assembly/target/scala-2.12/jars/jsr311-api-1.1.1.jar
```
And the log pointed out that the assembly phase copied it to `assembly/target/scala-2.12/jars/` which will be added to the classpath too. With the help of SBT `dependencyTree` tool, I saw the `jsr311-api` again as a transitive of `jersery-core` from `yarn` module with a `test` scope. So **This seems to be another bug from the SBT side of the `sbt-assembly` plugin.** It copied a test scope transitive artifact to the assembly output.
In this PR, I defined some rules in SparkBuild.scala to bypass the potential bugs from the SBT side.
First, exclude the `jsr311` from all over the project and then add it back separately to the YARN module for SBT.
Additionally, the HiveThriftServerSuites was reflected for reducing flakiness too, but not related to the bugs I have found so far.
### Why are the changes needed?
fix test here
### Does this PR introduce _any_ user-facing change?
NO
### How was this patch tested?
passing jenkins and ga
Closes#30643 from yaooqinn/HiveThriftHttpServerSuite.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Current Spark Web UI job page's header link of pending/skipped stages is inconsistent with their statuses. See the picture below:
![image](https://user-images.githubusercontent.com/9404831/101998894-1e843180-3c8c-11eb-8d94-10df9edb68e7.png)
### Why are the changes needed?
The code determining the `pendingOrSkippedTableId` has the wrong logic. As explained in the code:
> If the job is completed, then any pending stages are displayed as "skipped" [code pointer](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala#L266)
This PR fixes the logic for `pendingOrSkippedTableId` which aligns with the stage statuses.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Verified that header link is consistent with stage status with the fix.
Closes#30749 from linzebing/ui_bug.
Authored-by: linzebing <linzebing1995@gmail.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
### What changes were proposed in this pull request?
This PR proposes to change python to python3 in several places missed.
### Why are the changes needed?
To use Python 3 by default safely.
### Does this PR introduce _any_ user-facing change?
Yes, it will uses `python3` as its default Python interpreter.
### How was this patch tested?
It was tested together in https://github.com/apache/spark/pull/30735. The test cases there will verify this change together.
Closes#30750 from HyukjinKwon/SPARK-32447.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR aims to update `master` branch version to 3.2.0-SNAPSHOT.
### Why are the changes needed?
Start to prepare Apache Spark 3.2.0.
### Does this PR introduce _any_ user-facing change?
N/A.
### How was this patch tested?
Pass the CIs.
Closes#30606 from dongjoon-hyun/SPARK-3.2.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR proposes to make `spark.archives` configuration working in Kubernates.
It works without a problem in standalone cluster but there seems a bug in Kubernates.
It fails to fetch the file on the driver side as below:
```
20/12/03 13:33:53 INFO SparkContext: Added JAR file:/tmp/spark-75004286-c83a-4369-b624-14c5d2d2a748/spark-examples_2.12-3.1.0-SNAPSHOT.jar at spark://spark-test-app-48ae737628cee6f8-driver-svc.spark-integration-test.svc:7078/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar with timestamp 1607002432558
20/12/03 13:33:53 INFO SparkContext: Added archive file:///tmp/tmp4542734800151332666.txt.tar.gz#test_tar_gz at spark://spark-test-app-48ae737628cee6f8-driver-svc.spark-integration-test.svc:7078/files/tmp4542734800151332666.txt.tar.gz with timestamp 1607002432558
20/12/03 13:33:53 INFO TransportClientFactory: Successfully created connection to spark-test-app-48ae737628cee6f8-driver-svc.spark-integration-test.svc/172.17.0.4:7078 after 83 ms (47 ms spent in bootstraps)
20/12/03 13:33:53 INFO Utils: Fetching spark://spark-test-app-48ae737628cee6f8-driver-svc.spark-integration-test.svc:7078/files/tmp4542734800151332666.txt.tar.gz to /tmp/spark-66573e24-27a3-427c-99f4-36f06d9e9cd5/fetchFileTemp2665785666227461849.tmp
20/12/03 13:33:53 ERROR SparkContext: Error initializing SparkContext.
java.lang.RuntimeException: Stream '/files/tmp4542734800151332666.txt.tar.gz' was not found.
at org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:242)
at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:142)
at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:53)
```
This is because `spark.archives` was not actually added on the driver side correctly. The changes here fix it by adding and resolving URIs correctly.
### Why are the changes needed?
`spark.archives` feature can be leveraged for many things such as Conda support. We should make it working in Kubernates as well.
This is a bug fix too.
### Does this PR introduce _any_ user-facing change?
No, this feature is not out yet.
### How was this patch tested?
I manually tested with Minikube 1.15.1. For an environment issue (?), I had to use a custom namespace, service account and roles. `default` service account does not work for me and complains it doesn't have permissions to get/list pods, etc.
```bash
minikube delete
minikube start --cpus 12 --memory 16384
kubectl create namespace spark-integration-test
cat <<EOF | kubectl apply -f -
apiVersion: v1
kind: ServiceAccount
metadata:
name: spark
namespace: spark-integration-test
EOF
kubectl create clusterrolebinding spark-role --clusterrole=edit --serviceaccount=spark-integration-test:spark --namespace=spark-integration-test
dev/make-distribution.sh --pip --tgz -Pkubernetes
resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh --spark-tgz `pwd`/spark-3.1.0-SNAPSHOT-bin-3.2.0.tgz --service-account spark --namespace spark-integration-test
```
Closes#30581 from HyukjinKwon/SPARK-33615.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
`spark.buffer.size` not applied in driver from pyspark. In this PR I've fixed this issue.
### Why are the changes needed?
Apply the mentioned config on driver side.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing unit tests + manually.
Added the following code temporarily:
```
def local_connect_and_auth(port, auth_secret):
...
sock.connect(sa)
print("SPARK_BUFFER_SIZE: %d" % int(os.environ.get("SPARK_BUFFER_SIZE", 65536))) <- This is the addition
sockfile = sock.makefile("rwb", int(os.environ.get("SPARK_BUFFER_SIZE", 65536)))
...
```
Test:
```
#Compile Spark
echo "spark.buffer.size 10000" >> conf/spark-defaults.conf
$ ./bin/pyspark
Python 3.8.5 (default, Jul 21 2020, 10:48:26)
[Clang 11.0.3 (clang-1103.0.32.62)] on darwin
Type "help", "copyright", "credits" or "license" for more information.
20/12/03 13:38:13 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
20/12/03 13:38:14 WARN SparkEnv: I/O encryption enabled without RPC encryption: keys will be visible on the wire.
Welcome to
____ __
/ __/__ ___ _____/ /__
_\ \/ _ \/ _ `/ __/ '_/
/__ / .__/\_,_/_/ /_/\_\ version 3.1.0-SNAPSHOT
/_/
Using Python version 3.8.5 (default, Jul 21 2020 10:48:26)
Spark context Web UI available at http://192.168.0.189:4040
Spark context available as 'sc' (master = local[*], app id = local-1606999094506).
SparkSession available as 'spark'.
>>> sc.setLogLevel("TRACE")
>>> sc.parallelize([0, 2, 3, 4, 6], 5).glom().collect()
...
SPARK_BUFFER_SIZE: 10000
...
[[0], [2], [3], [4], [6]]
>>>
```
Closes#30592 from gaborgsomogyi/SPARK-33629.
Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
SPARK-9767 remove `ConnectionManager` and related files, the configuration `spark.core.connection.ack.wait.timeout` previously used by `ConnectionManager` is no longer used by other Spark code, but it still exists in the `configuration.md`.
So this pr cleans up the useless configuration item spark.core.connection.ack.wait.timeout` from `configuration.md`.
### Why are the changes needed?
Clean up useless configuration from `configuration.md`.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#30569 from LuciferYang/SPARK-33631.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
To make sure the sensitive attributes to be redacted in the history server log.
### Why are the changes needed?
We found the secure attributes like password in SparkListenerJobStart and SparkListenerStageSubmitted events would not been redated, resulting in sensitive attributes can be viewd directly.
The screenshot can be viewed in the attachment of JIRA spark-33504
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
muntual test works well, I have also added unit testcase.
Closes#30446 from akiyamaneko/eventlog_unredact.
Authored-by: neko <echohlne@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
### What changes were proposed in this pull request?
As described in SPARK-33557, `HeartbeatReceiver` and `MesosCoarseGrainedSchedulerBackend` will always use `Network.NETWORK_TIMEOUT.defaultValueString` as value of `STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT` when we configure `NETWORK_TIMEOUT` without configure `STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT`, this is different from the relationship described in `configuration.md`.
To fix this problem,the main change of this pr as follow:
- Remove the explicitly default value of `STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT`
- Use actual value of `NETWORK_TIMEOUT` as `STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT` when `STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT` not configured in `HeartbeatReceiver` and `MesosCoarseGrainedSchedulerBackend`
### Why are the changes needed?
To ensure the relationship between `NETWORK_TIMEOUT` and `STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT` as we described in `configuration.md`
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
- Pass the Jenkins or GitHub Action
- Manual test configure `NETWORK_TIMEOUT` and `STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT` locally
Closes#30547 from LuciferYang/SPARK-33557.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This reverts commit SPARK-33212 (cb3fa6c936) mostly with three exceptions:
1. `SparkSubmitUtils` was updated recently by SPARK-33580
2. `resource-managers/yarn/pom.xml` was updated recently by SPARK-33104 to add `hadoop-yarn-server-resourcemanager` test dependency.
3. Adjust `com.fasterxml.jackson.module:jackson-module-jaxb-annotations` dependency in K8s module which is updated recently by SPARK-33471.
### Why are the changes needed?
According to [HADOOP-16080](https://issues.apache.org/jira/browse/HADOOP-16080) since Apache Hadoop 3.1.1, `hadoop-aws` doesn't work with `hadoop-client-api`. It fails at write operation like the following.
**1. Spark distribution with `-Phadoop-cloud`**
```scala
$ bin/spark-shell --conf spark.hadoop.fs.s3a.access.key=$AWS_ACCESS_KEY_ID --conf spark.hadoop.fs.s3a.secret.key=$AWS_SECRET_ACCESS_KEY
20/11/30 23:01:24 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
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Spark context available as 'sc' (master = local[*], app id = local-1606806088715).
Spark session available as 'spark'.
Welcome to
____ __
/ __/__ ___ _____/ /__
_\ \/ _ \/ _ `/ __/ '_/
/___/ .__/\_,_/_/ /_/\_\ version 3.1.0-SNAPSHOT
/_/
Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_272)
Type in expressions to have them evaluated.
Type :help for more information.
scala> spark.read.parquet("s3a://dongjoon/users.parquet").show
20/11/30 23:01:34 WARN MetricsConfig: Cannot locate configuration: tried hadoop-metrics2-s3a-file-system.properties,hadoop-metrics2.properties
+------+--------------+----------------+
| name|favorite_color|favorite_numbers|
+------+--------------+----------------+
|Alyssa| null| [3, 9, 15, 20]|
| Ben| red| []|
+------+--------------+----------------+
scala> Seq(1).toDF.write.parquet("s3a://dongjoon/out.parquet")
20/11/30 23:02:14 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 2)/ 1]
java.lang.NoSuchMethodError: org.apache.hadoop.util.SemaphoredDelegatingExecutor.<init>(Lcom/google/common/util/concurrent/ListeningExecutorService;IZ)V
```
**2. Spark distribution without `-Phadoop-cloud`**
```scala
$ bin/spark-shell --conf spark.hadoop.fs.s3a.access.key=$AWS_ACCESS_KEY_ID --conf spark.hadoop.fs.s3a.secret.key=$AWS_SECRET_ACCESS_KEY -c spark.eventLog.enabled=true -c spark.eventLog.dir=s3a://dongjoon/spark-events/ --packages org.apache.hadoop:hadoop-aws:3.2.0,org.apache.hadoop:hadoop-common:3.2.0
...
java.lang.NoSuchMethodError: org.apache.hadoop.util.SemaphoredDelegatingExecutor.<init>(Lcom/google/common/util/concurrent/ListeningExecutorService;IZ)V
at org.apache.hadoop.fs.s3a.S3AFileSystem.create(S3AFileSystem.java:772)
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the CI.
Closes#30508 from dongjoon-hyun/SPARK-33212-REVERT.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
When running Spark behind a reverse proxy(e.g. Nginx, Apache HTTP server), the request URL can be encoded twice if we pass the query string directly to the constructor of `java.net.URI`:
```
> val uri = "http://localhost:8081/test"
> val query = "order%5B0%5D%5Bcolumn%5D=0" // query string of URL from the reverse proxy
> val rewrittenURI = URI.create(uri.toString())
> new URI(rewrittenURI.getScheme(),
rewrittenURI.getAuthority(),
rewrittenURI.getPath(),
query,
rewrittenURI.getFragment()).toString
result: http://localhost:8081/test?order%255B0%255D%255Bcolumn%255D=0
```
In Spark's stage page, the URL of "/taskTable" contains query parameter order[0][dir]. After encoding twice, the query parameter becomes `order%255B0%255D%255Bdir%255D` and it will be decoded as `order%5B0%5D%5Bdir%5D` instead of `order[0][dir]`. As a result, there will be NullPointerException from https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala#L176
Other than that, the other parameter may not work as expected after encoded twice.
This PR is to fix the bug by calling the method `URI.create(String URL)` directly. This convenience method can avoid encoding twice on the query parameter.
```
> val uri = "http://localhost:8081/test"
> val query = "order%5B0%5D%5Bcolumn%5D=0"
> URI.create(s"$uri?$query").toString
result: http://localhost:8081/test?order%5B0%5D%5Bcolumn%5D=0
> URI.create(s"$uri?$query").getQuery
result: order[0][column]=0
```
### Why are the changes needed?
Fix a potential bug when Spark's reverse proxy is enabled.
The bug itself is similar to https://github.com/apache/spark/pull/29271.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Add a new unit test.
Also, Manual UI testing for master, worker and app UI with an nginx proxy
Spark config:
```
spark.ui.port 8080
spark.ui.reverseProxy=true
spark.ui.reverseProxyUrl=/path/to/spark/
```
nginx config:
```
server {
listen 9000;
set $SPARK_MASTER http://127.0.0.1:8080;
# split spark UI path into prefix and local path within master UI
location ~ ^(/path/to/spark/) {
# strip prefix when forwarding request
rewrite /path/to/spark(/.*) $1 break;
#rewrite /path/to/spark/ "/" ;
# forward to spark master UI
proxy_pass $SPARK_MASTER;
proxy_intercept_errors on;
error_page 301 302 307 = handle_redirects;
}
location handle_redirects {
set $saved_redirect_location '$upstream_http_location';
proxy_pass $saved_redirect_location;
}
}
```
Closes#30552 from gengliangwang/decodeProxyRedirect.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
### What changes were proposed in this pull request?
TL;DR:
- This PR completes the support of archives in Spark itself instead of Yarn-only
- It makes `--archives` option work in other cluster modes too and adds `spark.archives` configuration.
- After this PR, PySpark users can leverage Conda to ship Python packages together as below:
```python
conda create -y -n pyspark_env -c conda-forge pyarrow==2.0.0 pandas==1.1.4 conda-pack==0.5.0
conda activate pyspark_env
conda pack -f -o pyspark_env.tar.gz
PYSPARK_DRIVER_PYTHON=python PYSPARK_PYTHON=./environment/bin/python pyspark --archives pyspark_env.tar.gz#environment
```
- Issue a warning that undocumented and hidden behavior of partial archive handling in `spark.files` / `SparkContext.addFile` will be deprecated, and users can use `spark.archives` and `SparkContext.addArchive`.
This PR proposes to add Spark's native `--archives` in Spark submit, and `spark.archives` configuration. Currently, both are supported only in Yarn mode:
```bash
./bin/spark-submit --help
```
```
Options:
...
Spark on YARN only:
--queue QUEUE_NAME The YARN queue to submit to (Default: "default").
--archives ARCHIVES Comma separated list of archives to be extracted into the
working directory of each executor.
```
This `archives` feature is useful often when you have to ship a directory and unpack into executors. One example is native libraries to use e.g. JNI. Another example is to ship Python packages together by Conda environment.
Especially for Conda, PySpark currently does not have a nice way to ship a package that works in general, please see also https://hyukjin-spark.readthedocs.io/en/stable/user_guide/python_packaging.html#using-zipped-virtual-environment (PySpark new documentation demo for 3.1.0).
The neatest way is arguably to use Conda environment by shipping zipped Conda environment but this is currently dependent on this archive feature. NOTE that we are able to use `spark.files` by relying on its undocumented behaviour that untars `tar.gz` but I don't think we should document such ways and promote people to more rely on it.
Also, note that this PR does not target to add the feature parity of `spark.files.overwrite`, `spark.files.useFetchCache`, etc. yet. I documented that this is an experimental feature as well.
### Why are the changes needed?
To complete the feature parity, and to provide a better support of shipping Python libraries together with Conda env.
### Does this PR introduce _any_ user-facing change?
Yes, this makes `--archives` works in Spark instead of Yarn-only, and adds a new configuration `spark.archives`.
### How was this patch tested?
I added unittests. Also, manually tested in standalone cluster, local-cluster, and local modes.
Closes#30486 from HyukjinKwon/native-archive.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR proposes to use current timestamp with warning log when the issue date for token is not set up properly. The next section will explain the rationalization with details.
### Why are the changes needed?
Unfortunately not every implementations respect the `issue date` in `AbstractDelegationTokenIdentifier`, which Spark relies on while calculating. The default value of issue date is 0L, which is far from actual issue date, breaking logic on calculating next renewal date under some circumstance, leading to 0 interval (immediate) on rescheduling token renewal.
In HadoopFSDelegationTokenProvider, Spark calculates token renewal interval as below:
2c64b731ae/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala (L123-L134)
The interval is calculated as `token.renew() - identifier.getIssueDate`, which is providing correct interval assuming both `token.renew()` and `identifier.getIssueDate` produce correct value, but it's going to be weird when `identifier.getIssueDate` provides 0L (default value), like below:
```
20/10/13 06:34:19 INFO security.HadoopFSDelegationTokenProvider: Renewal interval is 1603175657000 for token S3ADelegationToken/IDBroker
20/10/13 06:34:19 INFO security.HadoopFSDelegationTokenProvider: Renewal interval is 86400048 for token HDFS_DELEGATION_TOKEN
```
Hopefully we pick the minimum value as safety guard (so in this case, `86400048` is being picked up), but the safety guard leads unintentional bad impact on this case.
2c64b731ae/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala (L58-L71)
Spark leverages the interval being calculated in above, "minimum" value of intervals, and blindly adds the value to token's issue date to calculates the next renewal date for the token, and picks "minimum" value again. In problematic case, the value would be `86400048` (86400048 + 0) which is quite smaller than current timestamp.
2c64b731ae/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala (L228-L234)
The next renewal date is subtracted with current timestamp again to get the interval, and multiplexed by configured ratio to produce the final schedule interval. In problematic case, this value goes to negative.
2c64b731ae/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala (L180-L188)
There's a safety guard to not allow negative value, but that's simply 0 meaning schedule immediately. This triggers next calculation of next renewal date to calculate the schedule interval, lead to the same behavior, hence updating delegation token immediately and continuously.
As we fetch token just before the calculation happens, the actual issue date is likely slightly before, hence it's not that dangerous to use current timestamp as issue date for the token the issue date has not been set up properly. Still, it's better not to leave the token implementation as it is, so we log warn message to let end users consult with token implementer.
### Does this PR introduce _any_ user-facing change?
Yes. End users won't encounter the tight loop of schedule of token renewal after the PR. In end users' perspective of reflection, there's nothing end users need to change.
### How was this patch tested?
Manually tested with problematic environment.
Closes#30366 from HeartSaVioR/SPARK-33440.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
This PR aims to support storage migration to the fallback storage like cloud storage (`S3`) during worker decommission for the corner cases where the exceptions occur or there is no live peer left.
Although this PR focuses on cloud storage like `S3` which has a TTL feature in order to simplify Spark's logic, we can use alternative fallback storages like HDFS/NFS(EFS) if the user provides a clean-up mechanism.
### Why are the changes needed?
Currently, storage migration is not possible when there is no available executor. For example, when there is one executor, the executor cannot perform storage migration because it has no peer.
### Does this PR introduce _any_ user-facing change?
Yes. This is a new feature.
### How was this patch tested?
Pass the CIs with newly added test cases.
Closes#30492 from dongjoon-hyun/SPARK-33545.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
Fix some "hardcoded" API urls in Web UI.
More specifically, we avoid the use of `location.origin` when constructing URLs for internal API calls within the JavaScript.
Instead, we use `apiRoot` global variable.
### Why are the changes needed?
On one hand, it allows us to build relative URLs. On the other hand, `apiRoot` reflects the Spark property `spark.ui.proxyBase` which can be set to change the root path of the Web UI.
If `spark.ui.proxyBase` is actually set, original URLs become incorrect, and we end up with an executors blank page.
I encounter this bug when accessing the Web UI behind a proxy (in my case a Kubernetes Ingress).
See the following link for more context:
https://github.com/jupyterhub/jupyter-server-proxy/issues/57#issuecomment-699163115
### Does this PR introduce _any_ user-facing change?
Yes, as all the changes introduced are in the JavaScript for the Web UI.
### How the changes have been tested ?
I modified/debugged the JavaScript as in the commit with the help of the developer tools in Google Chrome, while accessing the Web UI of my Spark app behind my k8s ingress.
Closes#30523 from pgillet/fix-executors-blank-page-behind-proxy.
Authored-by: Pascal Gillet <pascal.gillet@stack-labs.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
### What changes were proposed in this pull request?
This PR intends to fix typos in the sub-modules:
* `bin`
* `core`
* `docs`
* `external`
* `mllib`
* `repl`
* `pom.xml`
Split per srowen https://github.com/apache/spark/pull/30323#issuecomment-728981618
NOTE: The misspellings have been reported at 706a726f87 (commitcomment-44064356)
### Why are the changes needed?
Misspelled words make it harder to read / understand content.
### Does this PR introduce _any_ user-facing change?
There are various fixes to documentation, etc...
### How was this patch tested?
No testing was performed
Closes#30530 from jsoref/spelling-bin-core-docs-external-mllib-repl.
Authored-by: Josh Soref <jsoref@users.noreply.github.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Currently we will kill the executor when hitting a fatal error. However, if the fatal error is wrapped by another exception, such as
- java.util.concurrent.ExecutionException, com.google.common.util.concurrent.UncheckedExecutionException, com.google.common.util.concurrent.ExecutionError when using Guava cache or Java thread pool.
- SparkException thrown from cf98a761de/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala (L231) or cf98a761de/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala (L296)
We will still keep the executor running. Fatal errors are usually unrecoverable (such as OutOfMemoryError), some components may be in a broken state when hitting a fatal error and it's hard to predicate the behaviors of a broken component. Hence, it's better to detect the nested fatal error as well and kill the executor. Then we can rely on Spark's fault tolerance to recover.
### Why are the changes needed?
Fatal errors are usually unrecoverable (such as OutOfMemoryError), some components may be in a broken state when hitting a fatal error and it's hard to predicate the behaviors of a broken component. Hence, it's better to detect the nested fatal error as well and kill the executor. Then we can rely on Spark's fault tolerance to recover.
### Does this PR introduce _any_ user-facing change?
Yep. There is a slight internal behavior change on when to kill an executor. We will kill the executor when detecting a nested fatal error in the exception chain. `spark.executor.killOnFatalError.depth` is added to allow users to turn off this change if the slight behavior change impacts them.
### How was this patch tested?
The new method `Executor.isFatalError` is tested by `spark.executor.killOnNestedFatalError`.
Closes#30528 from zsxwing/SPARK-33587.
Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This patch proposes to use classifier attribute to construct artifact path instead of type.
### Why are the changes needed?
`resolveDependencyPaths` now takes artifact type to decide to add "-tests" postfix. However, the path pattern of ivy in `resolveMavenCoordinates` is `[organization]_[artifact][revision](-[classifier]).[ext]`. We should use classifier instead of type to construct file path.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit test. Manual test.
Closes#30524 from viirya/SPARK-33580.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
1. Remove the fixed width style of class `container-fluid-div`. So that the UI looks clean when the text is long.
2. Add one space between a checkbox and the text on the right side, which is consistent with the stage page.
### Why are the changes needed?
The width of class `container-fluid-div` is set as 200px after https://github.com/apache/spark/pull/21688 . This makes the checkbox in the executor page messy.
![image](https://user-images.githubusercontent.com/1097932/100242069-3bc5ab80-2ee9-11eb-8c7d-96c221398fee.png)
We should remove the width limit.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test.
After the changes:
![image](https://user-images.githubusercontent.com/1097932/100257802-2f4a4e80-2efb-11eb-9eb0-92d6988ad14b.png)
Closes#30500 from gengliangwang/reviseStyle.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
When using dynamic partition overwrite, each task has its working dir under staging dir like `stagingDir/.spark-staging-{jobId}`, each task commits to `outputPath/.spark-staging-{jobId}/{partitionId}/part-{taskId}-{jobId}{ext}`.
When speculation enable, multiple task attempts would be setup for one task, **they have same task id and they would commit to same file concurrently**. Due to host done or node preemption, the partly-committed files aren't cleaned up, a FileAlreadyExistsException would be raised in this situation, resulting in job failure.
I don't try to change task commit process for dynamic partition overwrite, like adding attempt id to task working dir for each attempts and committing to final output dir via a new outputCommitCoordinator, here is reason:
1. `FileOutputCommitter` already has commit coordinator for each task attempts, we can leverage it rather than build a new one.
2. To say the least, we implement a coordinator solving task attempts commit conflict, suppose a severe case, application master failover, tasks with same attempt id and same task id would commit to same files, the `FileAlreadyExistsException` risk still exists
In this pr, I leverage FileOutputCommitter to solve the problem:
1. when initing a write job description, set `outputPath/.spark-staging-{jobId}` as the output dir
2. each task attempt writes output to `outputPath/.spark-staging-{jobId}/_temporary/${appAttemptId}/_temporary/${taskAttemptId}/{partitionId}/part-{taskId}-{jobId}{ext}`
3. leverage `FileOutputCommitter` coordinator, write job firstly commits output to `outputPath/.spark-staging-{jobId}/{partitionId}`
4. for dynamic partition overwrite, write job finally move `outputPath/.spark-staging-{jobId}/{partitionId}` to `outputPath/{partitionId}`
### Why are the changes needed?
Without this pr, dynamic partition overwrite would fail
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
added UT.
Closes#29000 from WinkerDu/master-fix-dynamic-partition-multi-commit.
Authored-by: duripeng <duripeng@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR aims the followings.
1. Upgrade from Scala 2.13.3 to 2.13.4 for Apache Spark 3.1
2. Fix exhaustivity issues in both Scala 2.12/2.13 (Scala 2.13.4 requires this for compilation.)
3. Enforce the improved exhaustive check by using the existing Scala 2.13 GitHub Action compilation job.
### Why are the changes needed?
Scala 2.13.4 is a maintenance release for 2.13 line and improves JDK 15 support.
- https://github.com/scala/scala/releases/tag/v2.13.4
Also, it improves exhaustivity check.
- https://github.com/scala/scala/pull/9140 (Check exhaustivity of pattern matches with "if" guards and custom extractors)
- https://github.com/scala/scala/pull/9147 (Check all bindings exhaustively, e.g. tuples components)
### Does this PR introduce _any_ user-facing change?
Yep. Although it's a maintenance version change, it's a Scala version change.
### How was this patch tested?
Pass the CIs and do the manual testing.
- Scala 2.12 CI jobs(GitHub Action/Jenkins UT/Jenkins K8s IT) to check the validity of code change.
- Scala 2.13 Compilation job to check the compilation
Closes#30455 from dongjoon-hyun/SCALA_3.13.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This reverts commit 065f17386d, which is not part of any released version. That is, this is an unreleased feature
### Why are the changes needed?
I like the concept of Trash, but I think this PR might just resolve a very specific issue by introducing a mechanism without a proper design doc. This could make the usage more complex.
I think we need to consider the big picture. Trash directory is an important concept. If we decide to introduce it, we should consider all the code paths of Spark SQL that could delete the data, instead of Truncate only. We also need to consider what is the current behavior if the underlying file system does not provide the API `Trash.moveToAppropriateTrash`. Is the exception good? How about the performance when users are using the object store instead of HDFS? Will it impact the GDPR compliance?
In sum, I think we should not merge the PR https://github.com/apache/spark/pull/29552 without the design doc and implementation plan. That is why I reverted it before the code freeze of Spark 3.1
### Does this PR introduce _any_ user-facing change?
Reverted the original commit
### How was this patch tested?
The existing tests.
Closes#30463 from gatorsmile/revertSpark-32481.
Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Spark creates local server to serialize several type of data for python. The python code tries to connect to the server, immediately after it's created but there are several system calls in between (this may change in each Spark version):
* getaddrinfo
* socket
* settimeout
* connect
Under some circumstances in heavy user environments these calls can be super slow (more than 15 seconds). These issues must be analyzed one-by-one but since these are system calls the underlying OS and/or DNS servers must be debugged and fixed. This is not trivial task and at the same time data processing must work somehow. In this PR I'm only intended to add a configuration possibility to increase the mentioned timeouts in order to be able to provide temporary workaround. The rootcause analysis is ongoing but I think this can vary in each case.
Because the server part doesn't contain huge amount of log entries to with one can measure time, I've added some.
### Why are the changes needed?
Provide workaround when localhost python server connection timeout appears.
### Does this PR introduce _any_ user-facing change?
Yes, new configuration added.
### How was this patch tested?
Existing unit tests + manual test.
```
#Compile Spark
echo "spark.io.encryption.enabled true" >> conf/spark-defaults.conf
echo "spark.python.authenticate.socketTimeout 10" >> conf/spark-defaults.conf
$ ./bin/pyspark
Python 3.8.5 (default, Jul 21 2020, 10:48:26)
[Clang 11.0.3 (clang-1103.0.32.62)] on darwin
Type "help", "copyright", "credits" or "license" for more information.
20/11/20 10:17:03 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
20/11/20 10:17:03 WARN SparkEnv: I/O encryption enabled without RPC encryption: keys will be visible on the wire.
Welcome to
____ __
/ __/__ ___ _____/ /__
_\ \/ _ \/ _ `/ __/ '_/
/__ / .__/\_,_/_/ /_/\_\ version 3.1.0-SNAPSHOT
/_/
Using Python version 3.8.5 (default, Jul 21 2020 10:48:26)
Spark context Web UI available at http://192.168.0.189:4040
Spark context available as 'sc' (master = local[*], app id = local-1605863824276).
SparkSession available as 'spark'.
>>> sc.setLogLevel("TRACE")
>>> sc.parallelize([0, 2, 3, 4, 6], 5).glom().collect()
20/11/20 10:17:09 TRACE PythonParallelizeServer: Creating listening socket
20/11/20 10:17:09 TRACE PythonParallelizeServer: Setting timeout to 10 sec
20/11/20 10:17:09 TRACE PythonParallelizeServer: Waiting for connection on port 59726
20/11/20 10:17:09 TRACE PythonParallelizeServer: Connection accepted from address /127.0.0.1:59727
20/11/20 10:17:09 TRACE PythonParallelizeServer: Client authenticated
20/11/20 10:17:09 TRACE PythonParallelizeServer: Closing server
...
20/11/20 10:17:10 TRACE SocketFuncServer: Creating listening socket
20/11/20 10:17:10 TRACE SocketFuncServer: Setting timeout to 10 sec
20/11/20 10:17:10 TRACE SocketFuncServer: Waiting for connection on port 59735
20/11/20 10:17:10 TRACE SocketFuncServer: Connection accepted from address /127.0.0.1:59736
20/11/20 10:17:10 TRACE SocketFuncServer: Client authenticated
20/11/20 10:17:10 TRACE SocketFuncServer: Closing server
[[0], [2], [3], [4], [6]]
>>>
```
Closes#30389 from gaborgsomogyi/SPARK-33143.
Lead-authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Revert the change in #29959 and don't remove `SerializableFileStatus` and `SerializableBlockLocation`.
### Why are the changes needed?
In Hadoop 2.7 `FileStatus` and `BlockLocation` are not serializable, so we still need the two wrapper classes.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
N/A
Closes#30447 from sunchao/SPARK-32381-followup.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
Driver side changes for coordinating push based shuffle by selecting external shuffle services for merging partitions.
This PR includes changes related to `ShuffleMapStage` preparation which is selection of merger locations and initializing them as part of `ShuffleDependency`.
Currently this code is not used as some of the changes would come subsequently as part of https://issues.apache.org/jira/browse/SPARK-32917 (shuffle blocks push as part of `ShuffleMapTask`), https://issues.apache.org/jira/browse/SPARK-32918 (support for finalize API) and https://issues.apache.org/jira/browse/SPARK-32920 (finalization of push/merge phase). This is why the tests here are also partial, once these above mentioned changes are raised as PR we will have enough tests for DAGScheduler piece of code as well.
### Why are the changes needed?
Added a new API in `SchedulerBackend` to get merger locations for push based shuffle. This is currently implemented for Yarn and other cluster managers can have separate implementations which is why a new API is introduced.
### Does this PR introduce _any_ user-facing change?
Yes, user facing config to enable push based shuffle is introduced
### How was this patch tested?
Added unit tests partially and some of the changes in DAGScheduler depends on future changes, DAGScheduler tests will be added along with those changes.
Lead-authored-by: Venkata krishnan Sowrirajan vsowrirajanlinkedin.com
Co-authored-by: Min Shen mshenlinkedin.com
Closes#30164 from venkata91/upstream-SPARK-32919.
Lead-authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Co-authored-by: Min Shen <mshen@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
### What changes were proposed in this pull request?
This pr add a new Scala compile arg to `pom.xml` to defense against new unused imports:
- `-Ywarn-unused-import` for Scala 2.12
- `-Wconf:cat=unused-imports:e` for Scala 2.13
The other fIles change are remove all unused imports in Spark code
### Why are the changes needed?
Cleanup code and add guarantee to defense against new unused imports
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#30351 from LuciferYang/remove-imports-core-module.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR is a follow-up of #29471 and does the following improvements for `HadoopFSUtils`:
1. Removes the extra `filterFun` from the listing API and combines it with the `filter`.
2. Removes `SerializableBlockLocation` and `SerializableFileStatus` given that `BlockLocation` and `FileStatus` are already serializable.
3. Hides the `isRootLevel` flag from the top-level API.
### Why are the changes needed?
Main purpose is to simplify the logic within `HadoopFSUtils` as well as cleanup the API.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Existing unit tests (e.g., `FileIndexSuite`)
Closes#29959 from sunchao/hadoop-fs-utils-followup.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
Supports python client deps from the launcher fs.
This is a feature that was added for java deps. This PR adds support fo rpythona s well.
yes
Manually running different scenarios and via examining the driver & executors logs. Also there is an integration test added.
I verified that the python resources are added to the spark file server and they are named properly so they dont fail the executors. Note here that as previously the following will not work:
primary resource `A.py`: uses a closure defined in submited pyfile `B.py`, context.py only adds to the pythonpath files with certain extension eg. zip, egg, jar.
Closes#25870 from skonto/python-deps.
Lead-authored-by: Stavros Kontopoulos <skontopo@redhat.com>
Co-authored-by: Stavros Kontopoulos <st.kontopoulos@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR aims to generalize executor metrics to support user-given file system schemes instead of the fixed `file,hdfs` scheme.
### Why are the changes needed?
For the users using only cloud storages like `S3A`, we need to be able to expose `S3A` metrics. Also, we can skip unused `hdfs` metrics.
### Does this PR introduce _any_ user-facing change?
Yes, but compatible for the existing users which uses `hdfs` and `file` filesystem scheme only.
### How was this patch tested?
Manually do the following.
```
$ build/sbt -Phadoop-cloud package
$ sbin/start-master.sh; sbin/start-slave.sh spark://$(hostname):7077
$ bin/spark-shell --master spark://$(hostname):7077 -c spark.executor.metrics.fileSystemSchemes=file,s3a -c spark.metrics.conf.executor.sink.jmx.class=org.apache.spark.metrics.sink.JmxSink
scala> spark.read.textFile("s3a://dongjoon/README.md").collect()
```
Separately, launch `jconsole` and check `*.executor.filesystem.s3a.*`. Also, confirm that there is no `*.executor.filesystem.hdfs.*`
```
$ jconsole
```
![Screen Shot 2020-11-17 at 9 26 03 PM](https://user-images.githubusercontent.com/9700541/99487609-94121180-291b-11eb-9ed2-964546146981.png)
Closes#30405 from dongjoon-hyun/SPARK-33476.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR proposes to simplify the exception messages from Python UDFS.
Currently, the exception message from Python UDFs is as below:
```python
from pyspark.sql.functions import udf; spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
process()
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
serializer.dump_stream(out_iter, outfile)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
self.serializer.dump_stream(self._batched(iterator), stream)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
for obj in iterator:
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
for item in iterator:
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
return lambda *a: f(*a)
File "/.../python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
return f(*args, **kwargs)
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
Actually, almost all cases, users only care about `ZeroDivisionError: division by zero`. We don't really have to show the internal stuff in 99% cases.
This PR adds a configuration `spark.sql.execution.pyspark.udf.simplifiedException.enabled` (disabled by default) that hides the internal tracebacks related to Python worker, (de)serialization, etc.
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
The trackback will be shown from the point when any non-PySpark file is seen in the traceback.
### Why are the changes needed?
Without this configuration. such internal tracebacks are exposed to users directly especially for shall or notebook users in PySpark. 99% cases people don't care about the internal Python worker, (de)serialization and related tracebacks. It just makes the exception more difficult to read. For example, one statement of `x/0` above shows a very long traceback and most of them are unnecessary.
This configuration enables the ability to show simplified tracebacks which users will likely be most interested in.
### Does this PR introduce _any_ user-facing change?
By default, no. It adds one configuration that simplifies the exception message. See the example above.
### How was this patch tested?
Manually tested:
```bash
$ pyspark --conf spark.sql.execution.pyspark.udf.simplifiedException.enabled=true
```
```python
from pyspark.sql.functions import udf; spark.sparkContext.setLogLevel("FATAL"); spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
and unittests were also added.
Closes#30309 from HyukjinKwon/SPARK-33407.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This adds support for Stage level scheduling to kubernetes. Kubernetes can support dynamic allocation via the shuffle tracking option which means we can support stage level scheduling by getting new executors.
The main changes here are having the k8s cluster manager pass the resource profile id into the executors and then the ExecutorsPodsAllocator has to request executors based on the individual resource profiles. I tried to keep code changes here to a minimum. I specifically choose to leave the ExecutorPodsSnapshot the way it was and construct the resource profile to pod states on the fly, with a fast path when not using other resource profiles, to keep the impact to a minimum. This results in the main changes required are just wrapping the allocation logic in a for loop over each profile. The other main change is in the basic feature step we have to look at the resources in the ResourceProfile to request pods with the correct resources. Much of the other logic like in the executor life cycle manager doesn't need to be resource profile.
This also adds support for [SPARK-32661]Spark executors on K8S should request extra memory for off-heap allocations because the stage level scheduling api has support for this and it made sense to make consistent with YARN. This was started with PR https://github.com/apache/spark/pull/29477 but never updated so I just did it here. To do this I moved a few functions around that were now used by both YARN and kubernetes so you will see some changes in Utils.
### Why are the changes needed?
Add the feature to Kubernetes based on customer feedback.
### Does this PR introduce _any_ user-facing change?
Yes the feature now works with K8s, but not underlying API changes.
### How was this patch tested?
Tested manually on kubernetes cluster and with unit tests.
Closes#30204 from tgravescs/stagek8sOrigSnapshotsRebase.
Lead-authored-by: Thomas Graves <tgraves@apache.org>
Co-authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
### What changes were proposed in this pull request?
Remove "in cluster mode" from the description of `spark.executor.memoryOverhead`
### Why are the changes needed?
fix correctness issue in documentaion
### Does this PR introduce _any_ user-facing change?
yes, users may not get confused about the description `spark.executor.memoryOverhead`
### How was this patch tested?
pass GA doc generation
Closes#30311 from yaooqinn/minordoc.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
1. Applies the SQL changes in SPARK-33230 to SparkHadoopWriter, so that `rdd.saveAsNewAPIHadoopDataset` passes in a unique job UUID in `spark.sql.sources.writeJobUUID`
1. `SparkHadoopWriterUtils.createJobTrackerID` generates a JobID by appending a random long number to the supplied timestamp to ensure the probability of a collision is near-zero.
1. With tests of uniqueness, round trips and negative jobID rejection.
### Why are the changes needed?
Without this, if more than one job is started in the same second *and the committer expects application attempt IDs to be unique* is at risk of clashing with other jobs.
With the fix,
* those committers which use the ID set in `spark.sql.sources.writeJobUUID` as a priority ID will pick that up instead and so be unique.
* committers which use the Hadoop JobID for unique paths and filenames will get the randomly generated jobID. Assuming all clocks in a cluster in sync, the probability of two jobs launched in the same second has dropped from 1 to 1/(2^63)
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Unit tests
There's a new test suite SparkHadoopWriterUtilsSuite which creates jobID, verifies they are unique even for the same timestamp and that they can be marshalled to string and parsed back in the hadoop code, which contains some (brittle) assumptions about the format of job IDs.
Functional Integration Tests
1. Hadoop-trunk built with [HADOOP-17318], publishing to local maven repository
1. Spark built with hadoop.version=3.4.0-SNAPSHOT to pick up these JARs.
1. Spark + Object store integration tests at [https://github.com/hortonworks-spark/cloud-integration](https://github.com/hortonworks-spark/cloud-integration) were built against that local spark version
1. And executed against AWS london.
The tests were run with `fs.s3a.committer.require.uuid=true`, so the s3a committers fail fast if they don't get a job ID down. This showed that `rdd.saveAsNewAPIHadoopDataset` wasn't setting the UUID option. It again uses the current Date value for an app attempt -which is not guaranteed to be unique.
With the change applied to spark, the relevant tests work, therefore the committers are getting unique job IDs.
Closes#30319 from steveloughran/BUG/SPARK-33402-jobuuid.
Authored-by: Steve Loughran <stevel@cloudera.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR aims to support sorted shuffle block migration.
### Why are the changes needed?
Since the current shuffle block migration works in a random order, the failure during worker decommission affects all shuffles. We had better finish the shuffles one by one to minimize the number of affected shuffle.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the CIs with the newly added test case.
Closes#30293 from dongjoon-hyun/SPARK-33387.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
There are two similar compilation warnings about procedure-like declaration in Scala 2.13:
```
[WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:70: procedure syntax is deprecated for constructors: add `=`, as in method definition
```
and
```
[WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala:211: procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `run`'s return type
```
this pr is the first part to resolve SPARK-33352:
- For constructors method definition add `=` to convert to function syntax
- For without `return type` methods definition add `: Unit =` to convert to function syntax
### Why are the changes needed?
Eliminate compilation warnings in Scala 2.13 and this change should be compatible with Scala 2.12
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#30255 from LuciferYang/SPARK-29392-FOLLOWUP.1.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
There are 4 fields in `MutableApplicationInfo ` seems useless:
- `coresGranted`
- `maxCores`
- `coresPerExecutor`
- `memoryPerExecutorMB`
They are always `None` and not reassigned.
So the main change of this pr is cleanup these useless fields in `MutableApplicationInfo`.
### Why are the changes needed?
Cleanup useless variables.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#30251 from LuciferYang/SPARK-33347.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
Add executor peak jvm memory metrics in executors page
![image](https://user-images.githubusercontent.com/1633312/97767765-9121bf00-1adb-11eb-93c7-7912d9fe7826.png)
### Why are the changes needed?
Users can know executor peak jvm metrics on in executors page
### Does this PR introduce _any_ user-facing change?
Users can know executor peak jvm metrics on in executors page
### How was this patch tested?
Manually tested
Closes#30186 from warrenzhu25/23432.
Authored-by: Warren Zhu <warren.zhu25@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
fix the wrong url and display name of blocking thread in threadDump page.
The blockingThreadId variable passed to the page should be of string type instead of Option type.
### Why are the changes needed?
blocking threadId in the ui page is not displayed well, and the corresponding url cannot be redirected normally
### Does this PR introduce _any_ user-facing change?
NO
### How was this patch tested?
The pr only involves minor changes to the page and does not affect other functions,
The manual test results are as follows. The thread name displayed on the page is correct, and you can click on the URL to jump to the corresponding url
![shows_ok](https://user-images.githubusercontent.com/52202080/98108177-89488d00-1ed6-11eb-9488-8446c3f38bad.gif)
Closes#30249 from akiyamaneko/thread-dump-improve.
Authored-by: neko <echohlne@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
### What changes were proposed in this pull request?
This PR proposes to register the executor source with the Spark metrics system when running in local mode.
### Why are the changes needed?
The Apache Spark metrics system provides many useful insights on the Spark workload.
In particular, the [executor source metrics](https://github.com/apache/spark/blob/master/docs/monitoring.md#component-instance--executor) provide detailed info, including the number of active tasks, I/O metrics, and several task metrics details. The executor source metrics, contrary to other sources (for example ExecutorMetrics source), is not available when running in local mode.
Having executor metrics in local mode can be useful when testing and troubleshooting Spark workloads in a development environment. The metrics can be fed to a dashboard to see the evolution of resource usage and can be used to troubleshoot performance,
as [in this example](https://github.com/cerndb/spark-dashboard).
Currently users will have to deploy on a cluster to be able to collect executor source metrics, while the possibility of having them in local mode is handy for testing.
### Does this PR introduce _any_ user-facing change?
- This PR exposes executor source metrics data when running in local mode.
### How was this patch tested?
- Manually tested by running in local mode and inspecting the metrics listed in http://localhost:4040/metrics/json/
- Also added a test in `SourceConfigSuite`
Closes#28528 from LucaCanali/metricsWithLocalMode.
Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Thomas Graves <tgraves@apache.org>
### What changes were proposed in this pull request?
In the old version of spark in the storage UI page, the sorting function is normal, but sorting in the new version will cause the header content to be lost, So I try to fix the bug.
### Why are the changes needed?
The header field of the table on the page is similar to the following, **note that each th contains the span attribute**:
```html
<thead>
<tr>
....
<th width="" class="">
<span data-toggle="tooltip" title="" data-original-title="StorageLevel displays where the persisted RDD is stored, format of the persisted RDD (serialized or de-serialized) andreplication factor of the persisted RDD">
Storage Level
</span>
</th>
.....
</tr>
</thead>
```
Since [PR#26136](https://github.com/apache/spark/pull/26136), if the `th` in the table itself contains the `span` attribute, the `span` will be deleted directly after clicking the sort, and the original header content will be lost.
There are three problems in `sorttable.js`:
1. `sortrevind.class = "sorttable_sortrevind"` in [sorttab.js#107](9d5e48ea95/core/src/main/resources/org/apache/spark/ui/static/sorttable.js (L107)) and `sortfwdind.class = "sorttable_sortfwdind"` in [sorttab.js#125](9d5e48ea95/core/src/main/resources/org/apache/spark/ui/static/sorttable.js (L125))
sorttable_xx attribute should be assigned to`className` instead of `class`, as javascript uses `rowlists[j].className.search` rather than `rowlists[j].class.search` to determine whether the component has a sorting flag or not.
2. `rowlists[j].className.search(/\sorttable_sortrevind\b/)` in [sorttab.js#120](9d5e48ea95/core/src/main/resources/org/apache/spark/ui/static/sorttable.js (L120)) was wrong. The original intention is to search whether `className` contains the word `sorttable_sortrevind` , but the expression is wrong, it should be `\bsorttable_sortrevind\b` instead of `\sorttable_sortrevind\b`
3. The if check statement in the following code snippet ([sorttab.js#141](9d5e48ea95/core/src/main/resources/org/apache/spark/ui/static/sorttable.js (L141))) was wrong. **If the `search` function does not find the target, it will return -1, but Boolean(-1) is actually equals true**. This statement will cause span to be deleted even if it does not contain `sorttable_sortfwdind` or `sorttable_sortrevind`.
```javascript
rowlists = this.parentNode.getElementsByTagName("span");
for (var j=0; j < rowlists.length; j++) {
if (rowlists[j].className.search(/\bsorttable_sortfwdind\b/)
|| rowlists[j].className.search(/\sorttable_sortrevind\b/) ) {
rowlists[j].parentNode.removeChild(rowlists[j]);
}
}
```
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
The manual test result of the ui page is as below:
![fix sorted](https://user-images.githubusercontent.com/52202080/97543194-daeaa680-1a02-11eb-8b11-8109c3e4e9a3.gif)
Closes#30182 from akiyamaneko/ui_storage_sort_error.
Authored-by: neko <echohlne@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Allow to run the Spark web UI behind a reverse proxy with URLs prefixed by a context root, like www.mydomain.com/spark. In particular, this allows to access multiple Spark clusters through the same virtual host, only distinguishing them by context root, like www.mydomain.com/cluster1, www.mydomain.com/cluster2, and it allows to run the Spark UI in a common cookie domain (for SSO) with other services.
### Why are the changes needed?
This PR is to take over https://github.com/apache/spark/pull/17455.
After changes, Spark allows showing customized prefix URL in all the `href` links of the HTML pages.
### Does this PR introduce _any_ user-facing change?
Yes, all the links of UI pages will be contains the value of `spark.ui.reverseProxyUrl` if it is configurated.
### How was this patch tested?
New HTML Unit tests in MasterSuite
Manual UI testing for master, worker and app UI with an nginx proxy
Spark config:
```
spark.ui.port 8080
spark.ui.reverseProxy=true
spark.ui.reverseProxyUrl=/path/to/spark/
```
nginx config:
```
server {
listen 9000;
set $SPARK_MASTER http://127.0.0.1:8080;
# split spark UI path into prefix and local path within master UI
location ~ ^(/path/to/spark/) {
# strip prefix when forwarding request
rewrite /path/to/spark(/.*) $1 break;
#rewrite /path/to/spark/ "/" ;
# forward to spark master UI
proxy_pass $SPARK_MASTER;
proxy_intercept_errors on;
error_page 301 302 307 = handle_redirects;
}
location handle_redirects {
set $saved_redirect_location '$upstream_http_location';
proxy_pass $saved_redirect_location;
}
}
```
Closes#29820 from gengliangwang/revertProxyURL.
Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Oliver Köth <okoeth@de.ibm.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
### What changes were proposed in this pull request?
this PR renames the blacklisting feature. I ended up using "excludeOnFailure" or "excluded" in most cases but there is a mix. I renamed the BlacklistTracker to HealthTracker, but for the TaskSetBlacklist HealthTracker didn't make sense to me since its not the health of the taskset itself but rather tracking the things its excluded on so I renamed it to be TaskSetExcludeList. Everything else I tried to use the context and in most cases excluded made sense. It made more sense to me then blocked since you are basically excluding those executors and nodes from scheduling tasks on them. Then can be unexcluded later after timeouts and such. The configs I changed the name to use excludeOnFailure which I thought explained it.
I unfortunately couldn't get rid of some of them because its part of the event listener and history files. To keep backwards compatibility I kept the events and some of the parsing so that the history server would still properly read older history files. It is not forward compatible though - meaning a new application write the "Excluded" events so the older history server won't properly read display them as being blacklisted.
A few of the files below are showing up as deleted and recreated even though I did a git mv on them. I'm not sure why.
### Why are the changes needed?
get rid of problematic language
### Does this PR introduce _any_ user-facing change?
Config name changes but the old configs still work but are deprecated.
### How was this patch tested?
updated tests and also manually tested the UI changes and manually tested the history server reading older versions of history files and vice versa.
Closes#29906 from tgravescs/SPARK-32037.
Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Signed-off-by: Thomas Graves <tgraves@apache.org>
### What changes were proposed in this pull request?
Small typo fix in the description of `spark.storage.decommission.shuffleBlocks.enabled` property.
Closes#30208 from dsabanin/patch-1.
Authored-by: Dmitry Sabanin <sdmitry@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
"external block store" API was removed after SPARK-12667, `externalBlockStoreSize` in `RDDInfo` looks like always 0 and useless. So this pr just to remove this useless variable.
### Why are the changes needed?
remove useless variable.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#30179 from LuciferYang/SPARK-12667-FOLLOWUP.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This patch separates the view permission checks from the getAppUi in FsHistoryServerProvider, thus enabling SHS to do view permissions check of a given attempt for a given user without rebuilding the UI. This is achieved by adding a method "checkUIViewPermissions(appId: String, attemptId: Option[String], user: String): Boolean" to many layers of history server components. Currently, this feature is useful for event log download.
### Why are the changes needed?
Right now, when we want to download the event logs from the spark history server, SHS will need to parse entire the event log to rebuild UI, and this is just for view permission checks. UI rebuilding is a time-consuming and memory-intensive task, especially for large logs. However, this process is unnecessary for event log download. With this patch, UI rebuild can be skipped when downloading event logs from the history server. Thus the time of downloading a GB scale event log can be reduced from several minutes to several seconds, and the memory consumption of UI rebuilding can be avoided.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added test cases to confirm the view permission checks work properly and download event logs won't trigger UI loading. Also did some manual tests to verify the download speed can be drastically improved and the authentication works properly.
Closes#30126 from baohe-zhang/bypass_ui_rebuild_for_log_download.
Authored-by: Baohe Zhang <baohe.zhang@verizonmedia.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
The page returned by /jobs in Spark UI will store the detail information of each job in javascript like this:
```javascript
{
'className': 'executor added',
'group': 'executors',
'start': new Date(1602834008978),
'content': '<div class="executor-event-content"' +
'data-toggle="tooltip" data-placement="top"' +
'data-title="Executor 3<br>' +
'Added at 2020/10/16 15:40:08"' +
'data-html="true">Executor 3 added</div>'
}
```
if an application has a failed job, the failure reason corresponding to the job will be stored in the ` content` field in the javascript . if the failure reason contains the character: **'**, the javascript code will throw an exception to cause the `event timeline url` had no response , The following is an example of error json:
```javascript
{
'className': 'executor removed',
'group': 'executors',
'start': new Date(1602925908654),
'content': '<div class="executor-event-content"' +
'data-toggle="tooltip" data-placement="top"' +
'data-title="Executor 2<br>' +
'Removed at 2020/10/17 17:11:48' +
'<br>Reason: Container from a bad node: ... 20/10/17 16:00:42 WARN ShutdownHookManager: ShutdownHook **'$anon$2'** timeout..."' +
'data-html="true">Executor 2 removed</div>'
}
```
So we need to considier this special case , if the returned job info contains the character:**'**, just remove it
### Why are the changes needed?
Ensure that the UI page can function normally
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
This pr only fixes an exception in a special case, manual test result as blows:
![fixed](https://user-images.githubusercontent.com/52202080/96711638-74490580-13d0-11eb-93e0-b44d9ed5da5c.gif)
Closes#30119 from akiyamaneko/timeline_view_cannot_open.
Authored-by: neko <echohlne@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
### What changes were proposed in this pull request?
This PR cleans up the RPC message flow among the multiple decommission use cases, it includes changes:
* Keep `Worker`'s decommission status be consistent between the case where decommission starts from `Worker` and the case where decommission starts from the `MasterWebUI`: sending `DecommissionWorker` from `Master` to `Worker` in the latter case.
* Change from two-way communication to one-way communication when notifying decommission between driver and executor: it's obviously unnecessary for the executor to acknowledge the decommission status to the driver since the decommission request is from the driver. And it's same in reverse.
* Only send one message instead of two(`DecommissionSelf`/`DecommissionBlockManager`) when decommission the executor: executor and `BlockManager` are in the same JVM.
* Clean up codes around here.
### Why are the changes needed?
Before:
<img width="1948" alt="WeChat56c00cc34d9785a67a544dca036d49da" src="https://user-images.githubusercontent.com/16397174/92850308-dc461c80-f41e-11ea-8ac0-287825f4e0c4.png">
After:
<img width="1968" alt="WeChat05f7afb017e3f0132394c5e54245e49e" src="https://user-images.githubusercontent.com/16397174/93189571-de88dd80-f774-11ea-9300-1943920aa27d.png">
(Note the diagrams only counts those RPC calls that needed to go through the network. Local RPC calls are not counted here.)
After this change, We reduced 6 original RPC calls and added one more RPC call for keeping the consistent decommission status for the Worker. And the RPC flow becomes more clear.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Updated existing tests.
Closes#29817 from Ngone51/simplify-decommission-rpc.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
- Building spark internally in orgs where access to outside internet is not allowed takes a long time because unsuccessful attempts are made to download artifacts from repositories which are not accessible. The unsuccessful attempts unnecessarily add significant amount of time to the build. I have seen a difference of up-to 1hr for some runs.
- Adding 1 environment variables that should be present that the start of the build and if they exist, override the default repos defined in the code and scripts.
envVariables:
- DEFAULT_ARTIFACT_REPOSITORY=https://artifacts.internal.com/libs-release/
### Why are the changes needed?
To allow orgs to build spark internally without relying on external repositories for artifact downloads.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Multiple builds with and without env variables set.
Closes#29874 from ankits/SPARK-32998.
Authored-by: Ankit Srivastava <ankit_srivastava@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This updates the misleading log messages for removed shuffle block during migration.
### Why are the changes needed?
1. For the deleted shuffle blocks, `IndexShuffleBlockResolver` shows users WARN message saying `skipping migration`. However, `BlockManagerDecommissioner` shows users INFO message including `Migrated ShuffleBlockInfo(...)` inconsistently. Technically, we didn't migrated. We should not show `Migrated` message in this case.
```
INFO BlockManagerDecommissioner: Trying to migrate shuffle ShuffleBlockInfo(109,18924) to BlockManagerId(...) (2 / 3)
WARN IndexShuffleBlockResolver: Failed to resolve shuffle block ShuffleBlockInfo(109,18924), skipping migration. This is expected to occur if a block is removed after decommissioning has started.
INFO BlockManagerDecommissioner: Got migration sub-blocks List()
...
INFO BlockManagerDecommissioner: Migrated ShuffleBlockInfo(109,18924) to BlockManagerId(...)
```
2. In addition, if the shuffle file is deleted while the information is in the queue, the above messages are repeated multiple times, `spark.storage.decommission.maxReplicationFailuresPerBlock`. We had better use one line instead of the group of messages for that case.
```
INFO BlockManagerDecommissioner: Trying to migrate shuffle ShuffleBlockInfo(109,18924) to BlockManagerId(...) (0 / 3)
...
INFO BlockManagerDecommissioner: Trying to migrate shuffle ShuffleBlockInfo(109,18924) to BlockManagerId(...) (1 / 3)
...
INFO BlockManagerDecommissioner: Trying to migrate shuffle ShuffleBlockInfo(109,18924) to BlockManagerId(...) (2 / 3)
```
3. Skipping or not is a role of `BlockManagerDecommissioner` class. `IndexShuffleBlockResolver.getMigrationBlocks` is used twice differently like the following. We had better inform users at `BlockManagerDecommissioner` once.
- At the beginning, to get the sub-blocks.
- In case of `IOException`, to determine whether ignoring it or re-throwing. And, `BlockManagerDecommissioner` shows WARN message (`Skipping block ...`) again.
### Does this PR introduce _any_ user-facing change?
No. This is an update for log message info to be consistent.
### How was this patch tested?
Manually.
Closes#30129 from dongjoon-hyun/SPARK-33218.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This switches Spark to use shaded Hadoop clients, namely hadoop-client-api and hadoop-client-runtime, for Hadoop 3.x. For Hadoop 2.7, we'll still use the same modules such as hadoop-client.
In order to still keep default Hadoop profile to be hadoop-3.2, this defines the following Maven properties:
```
hadoop-client-api.artifact
hadoop-client-runtime.artifact
hadoop-client-minicluster.artifact
```
which default to:
```
hadoop-client-api
hadoop-client-runtime
hadoop-client-minicluster
```
but all switch to `hadoop-client` when the Hadoop profile is hadoop-2.7. A side affect from this is we'll import the same dependency multiple times. For this I have to disable Maven enforcer `banDuplicatePomDependencyVersions`.
Besides above, there are the following changes:
- explicitly add a few dependencies which are imported via transitive dependencies from Hadoop jars, but are removed from the shaded client jars.
- removed the use of `ProxyUriUtils.getPath` from `ApplicationMaster` which is a server-side/private API.
- modified `IsolatedClientLoader` to exclude `hadoop-auth` jars when Hadoop version is 3.x. This change should only matter when we're not sharing Hadoop classes with Spark (which is _mostly_ used in tests).
### Why are the changes needed?
This serves two purposes:
- to unblock Spark from upgrading to Hadoop 3.2.2/3.3.0+. Latest Hadoop versions have upgraded to use Guava 27+ and in order to adopt the latest Hadoop versions in Spark, we'll need to resolve the Guava conflicts. This takes the approach by switching to shaded client jars provided by Hadoop.
- avoid pulling 3rd party dependencies from Hadoop and avoid potential future conflicts.
### Does this PR introduce _any_ user-facing change?
When people use Spark with `hadoop-provided` option, they should make sure class path contains `hadoop-client-api` and `hadoop-client-runtime` jars. In addition, they may need to make sure these jars appear before other Hadoop jars in the order. Otherwise, classes may be loaded from the other non-shaded Hadoop jars and cause potential conflicts.
### How was this patch tested?
Relying on existing tests.
Closes#29843 from sunchao/SPARK-29250.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
### What changes were proposed in this pull request?
This PR changes `<` into `>` in the following to fix data loss during storage migrations.
```scala
// If we found any new shuffles to migrate or otherwise have not migrated everything.
- newShufflesToMigrate.nonEmpty || migratingShuffles.size < numMigratedShuffles.get()
+ newShufflesToMigrate.nonEmpty || migratingShuffles.size > numMigratedShuffles.get()
```
### Why are the changes needed?
`refreshOffloadingShuffleBlocks` should return `true` when the migration is still on-going.
Since `migratingShuffles` is defined like the following, `migratingShuffles.size > numMigratedShuffles.get()` means the migration is not finished.
```scala
// Shuffles which are either in queue for migrations or migrated
protected[storage] val migratingShuffles = mutable.HashSet[ShuffleBlockInfo]()
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the CI with the updated test cases.
Closes#30116 from dongjoon-hyun/SPARK-33202.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR aims to fix `getMigrationBlocks` error handling and to add test coverage.
1. `getMigrationBlocks` should not fail at indexFile only case.
2. `assert` causes `java.lang.AssertionError` which is not an `Exception`.
### Why are the changes needed?
To handle the exception correctly.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the CI with the newly added test case.
Closes#30110 from dongjoon-hyun/SPARK-33198.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR aims to improve the log message for better analysis.
### Why are the changes needed?
Good logs are crucial always.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual review.
Closes#30109 from dongjoon-hyun/k8s_log.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR proposes to fix:
```
org.apache.spark.deploy.yarn.YarnClusterSuite.run Python application in yarn-client mode
org.apache.spark.deploy.yarn.YarnClusterSuite.run Python application in yarn-cluster mode
org.apache.spark.deploy.yarn.YarnClusterSuite.run Python application in yarn-cluster mode using spark.yarn.appMasterEnv to override local envvar
```
it currently fails as below:
```
20/10/16 19:20:36 WARN TaskSetManager: Lost task 0.0 in stage 0.0 (TID 0) (amp-jenkins-worker-03.amp executor 1): org.apache.spark.SparkException:
Error from python worker:
Traceback (most recent call last):
File "/usr/lib64/python2.6/runpy.py", line 104, in _run_module_as_main
loader, code, fname = _get_module_details(mod_name)
File "/usr/lib64/python2.6/runpy.py", line 79, in _get_module_details
loader = get_loader(mod_name)
File "/usr/lib64/python2.6/pkgutil.py", line 456, in get_loader
return find_loader(fullname)
File "/usr/lib64/python2.6/pkgutil.py", line 466, in find_loader
for importer in iter_importers(fullname):
File "/usr/lib64/python2.6/pkgutil.py", line 422, in iter_importers
__import__(pkg)
File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/pyspark/__init__.py", line 53, in <module>
from pyspark.rdd import RDD, RDDBarrier
File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/pyspark/rdd.py", line 34, in <module>
from pyspark.java_gateway import local_connect_and_auth
File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/pyspark/java_gateway.py", line 29, in <module>
from py4j.java_gateway import java_import, JavaGateway, JavaObject, GatewayParameters
File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 60
PY4J_TRUE = {"yes", "y", "t", "true"}
^
SyntaxError: invalid syntax
```
I think this was broken when Python 2 was dropped but was not caught because this specific test does not run when there's no change in YARN codes. See also https://github.com/apache/spark/pull/29843#issuecomment-712540024
The root cause seems like the paths are different, see https://github.com/apache/spark/pull/29843#pullrequestreview-502595199. I _think_ Jenkins uses a different Python executable via Anaconda and the executor side does not know where it is for some reasons.
This PR proposes to fix it just by explicitly specifying the absolute path for Python executable so the tests should pass in any environment.
### Why are the changes needed?
To make tests pass.
### Does this PR introduce _any_ user-facing change?
No, dev-only.
### How was this patch tested?
This issue looks specific to Jenkins. It should run the tests on Jenkins.
Closes#30099 from HyukjinKwon/SPARK-33191.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Improve error message on reading unexpected directory
### Why are the changes needed?
Improve error message on reading unexpected directory
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Ut
Closes#30027 from AngersZhuuuu/SPARK-32069.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR aims to use `eventually` to fix the flakiness of the test case `SPARK-33088: executor failed tasks trigger plugin calls`.
### Why are the changes needed?
The test case checks like the following.
```scala
assert(TestSparkPlugin.executorPlugin.numOnTaskStart == 2)
assert(TestSparkPlugin.executorPlugin.numOnTaskSucceeded == 0)
assert(TestSparkPlugin.executorPlugin.numOnTaskFailed == 2)
```
Although first and second passed, the third can fail.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-hive-2.3-jdk-11/lastCompletedBuild/testReport/org.apache.spark.internal.plugin/PluginContainerSuite/SPARK_33088__executor_failed_tasks_trigger_plugin_calls/
- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129919/testReport/
```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException: 1 did not equal 2
at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
at org.scalatest.Assertions$.newAssertionFailedException(Assertions.scala:1231)
at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:1295)
at org.apache.spark.internal.plugin.PluginContainerSuite.$anonfun$new$8(PluginContainerSuite.scala:161)
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
This only improves the robustness.
Closes#30072 from dongjoon-hyun/SPARK-33173.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
If a block is removed between discovery to transfer fo the block, we short circuit that block and remove it from the list to transfer and increment the transferred blocks. This is complicated since both RPC errors and local read errors may be reported with the same exception class.
### Why are the changes needed?
Slow shuffle refreshes could waste time when decommissioning has already finished. Decommissioning might avoid transferring some some blocks to an otherwise live host which is marked as "full" if a deleted block fails to transfer to that host.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
New unit and integration tests.
Closes#30046 from holdenk/handle-cleaned-shuffles-during0migration.
Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR proposes to use `scala.Predef.assert` instead of `org.scalatest.Assertions.assert` removed at https://github.com/apache/spark/pull/30064
### Why are the changes needed?
Just to keep the same behaviour.
### Does this PR introduce _any_ user-facing change?
No, dev-only
### How was this patch tested?
Recover the existing asserts.
Closes#30065 from HyukjinKwon/SPARK-33165.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Proposing a new set of APIs for ExecutorPlugins, to provide callbacks invoked at the start and end of each task of a job. Not very opinionated on the shape of the API, tried to be as minimal as possible for now.
### Why are the changes needed?
Changes described in detail on [SPARK-33088](https://issues.apache.org/jira/browse/SPARK-33088), but mostly this boils down to:
1. This feature was considered when the ExecutorPlugin API was initially introduced in #21923, but never implemented.
2. The use-case which **requires** this feature is to propagate tracing information from the driver to the executor, such that calls from the same job can all be traced.
a. Tracing frameworks usually are setup in thread locals, therefore it's important for the setup to happen in the same thread which runs the tasks.
b. Executors can be for multiple jobs, therefore it's not sufficient to set tracing information at executor startup time -- it needs to happen every time a task starts or ends.
### Does this PR introduce _any_ user-facing change?
No. This PR introduces new features for future developers to use.
### How was this patch tested?
Unit tests on `PluginContainerSuite`.
Closes#29977 from fsamuel-bs/SPARK-33088.
Authored-by: Samuel Souza <ssouza@palantir.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
### What changes were proposed in this pull request?
This PR proposes to remove `assert` from `Benchmark` for making it easier to run benchmark codes via `spark-submit`.
### Why are the changes needed?
Since the current `Benchmark` (`master` and `branch-3.0`) has `assert`, we need to pass the proper jars of `scalatest` and `scalactic`;
- scalatest-core_2.12-3.2.0.jar
- scalatest-compatible-3.2.0.jar
- scalactic_2.12-3.0.jar
```
./bin/spark-submit --jars scalatest-core_2.12-3.2.0.jar,scalatest-compatible-3.2.0.jar,scalactic_2.12-3.0.jar,./sql/catalyst/target/spark-catalyst_2.12-3.1.0-SNAPSHOT-tests.jar,./core/target/spark-core_2.12-3.1.0-SNAPSHOT-tests.jar --class org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmark ./sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar --data-location /tmp/tpcds-sf1
```
This update can make developers submit benchmark codes without these dependencies;
```
./bin/spark-submit --jars ./sql/catalyst/target/spark-catalyst_2.12-3.1.0-SNAPSHOT-tests.jar,./core/target/spark-core_2.12-3.1.0-SNAPSHOT-tests.jar --class org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmark ./sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar --data-location /tmp/tpcds-sf1
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manually checked.
Closes#30064 from maropu/RemoveDepInBenchmark.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Current solution in build file to enable build failure on compilation warnings with exclusion of deprecation ones is not portable after SBT version 1.3.13 (build import fails with compilation error with SBT 1.4) and could be replaced with more robust and maintainable, especially since Scala 2.13.2 with similar built-in functionality.
Additionally, warnings were fixed to pass the build, with as few changes as possible:
warnings in 2.12 compilation fixed in code,
warnings in 2.13 compilation covered by configuration to be addressed separately
### Why are the changes needed?
Unblocks upgrade to SBT after 1.3.13.
Enhances build file maintainability.
Allows fine tune of warnings configuration in scope of Scala 2.13 compilation.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`build/sbt`'s `compile` and `Test/compile` for both Scala 2.12 and 2.13 profiles.
Closes#29995 from gemelen/feature/warnings-reporter.
Authored-by: Denis Pyshev <git@gemelen.net>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
This is the first patch for SPIP SPARK-30602 for push-based shuffle.
Summary of changes:
* Introduce new API in ExternalBlockStoreClient to push blocks to a remote shuffle service.
* Leveraging the streaming upload functionality in SPARK-6237, it also enables the ExternalBlockHandler to delegate the handling of block push requests to MergedShuffleFileManager.
* Propose the API for MergedShuffleFileManager, where the core logic on the shuffle service side to handle block push requests is defined. The actual implementation of this API is deferred into a later RB to restrict the size of this PR.
* Introduce OneForOneBlockPusher to enable pushing blocks to remote shuffle services in shuffle RPC layer.
* New protocols in shuffle RPC layer to support the functionalities.
### Why are the changes needed?
Refer to the SPIP in 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.
We have already verified the functionality and the improved performance as documented in the SPIP doc.
Lead-authored-by: Min Shen <mshenlinkedin.com>
Co-authored-by: Chandni Singh <chsinghlinkedin.com>
Co-authored-by: Ye Zhou <yezhoulinkedin.com>
Closes#29855 from Victsm/SPARK-32915.
Lead-authored-by: Min Shen <mshen@linkedin.com>
Co-authored-by: Chandni Singh <chsingh@linkedin.com>
Co-authored-by: Ye Zhou <yezhou@linkedin.com>
Co-authored-by: Chandni Singh <singh.chandni@gmail.com>
Co-authored-by: Min Shen <victor.nju@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
### What changes were proposed in this pull request?
This PR aims to ignore Apache Spark 2.4.x distribution in HiveExternalCatalogVersionsSuite if Python version is 3.8 or 3.9.
### Why are the changes needed?
Currently, `HiveExternalCatalogVersionsSuite` is broken on the latest OS like `Ubuntu 20.04` because its default Python version is 3.8. PySpark 2.4.x doesn't work on Python 3.8 due to SPARK-29536.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manually.
```
$ python3 --version
Python 3.8.5
$ build/sbt "hive/testOnly *.HiveExternalCatalogVersionsSuite"
...
[info] All tests passed.
[info] Passed: Total 1, Failed 0, Errors 0, Passed 1
```
Closes#30044 from dongjoon-hyun/SPARK-33153.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Adds an additional check for non-fatal errors when attempting to add a new entry to the history server application listing.
### Why are the changes needed?
A bad rolling event log folder (missing appstatus file or no log files) would cause no applications to be loaded by the Spark history server. Figuring out why invalid event log folders are created in the first place will be addressed in separate issues, this just lets the history server skip the invalid folder and successfully load all the valid applications.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
New UT
Closes#30037 from Kimahriman/bug/rolling-log-crashing-history.
Authored-by: Adam Binford <adam.binford@radiantsolutions.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
when bytesRead metric was negative, `formatBytes` in `ui.js` should just return `0.0 B` to avoid `NaN Undefined` result.
### Why are the changes needed?
Strengthen the parameter validataion to improve metric display on Summary Metrics of Spark Stage UI.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
It's a small change, just manual test.
Closes#30030 from akiyamaneko/formatBytes_NaN.
Authored-by: neko <echohlne@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Decommissioning can run out of time resulting in some race condition, these race conditions result in confusing error messages but not negative impact.
### Why are the changes needed?
The NPE & element missing errors in the log can create a missunderstanding.
### Does this PR introduce _any_ user-facing change?
Logs change.
### How was this patch tested?
Existing tests pass.
Closes#29992 from holdenk/SPARK-32881-error-messaging-on-decom-race-messages.
Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Fix the flaky test.
### Why are the changes needed?
The test is flaky: `Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown`.
Check the full error stack [here](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128548/testReport/org.apache.spark.scheduler/BarrierTaskContextSuite/throw_exception_if_the_number_of_barrier___calls_are_not_the_same_on_every_task/).
By analyzing the log below, I found that task 0 hadn't reached the second `context.barrier()` when another three tasks already raised the sync timeout exceptions by the first `context.barrier()`. The timeout exceptions were caught by the `try...catch...`. Then, each task started another round barrier sync from the second `context.barrier()` and completed the sync successfully.
```scala
20/09/10 20:54:48.821 dispatcher-event-loop-10 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:48.822 dispatcher-event-loop-10 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 2, current progress: 1/4.
20/09/10 20:54:48.826 dispatcher-BlockManagerMaster INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on localhost:38420 (size: 2.2 KiB, free: 546.3 MiB)
20/09/10 20:54:48.908 dispatcher-event-loop-12 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:48.909 dispatcher-event-loop-12 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 1, current progress: 2/4.
20/09/10 20:54:48.959 dispatcher-event-loop-11 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:48.960 dispatcher-event-loop-11 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 3, current progress: 3/4.
20/09/10 20:54:49.616 dispatcher-CoarseGrainedScheduler INFO TaskSchedulerImpl: Skip current round of resource offers for barrier stage 0 because the barrier taskSet requires 4 slots, while the total number of available slots is 0.
20/09/10 20:54:49.899 dispatcher-event-loop-15 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:49.900 dispatcher-event-loop-15 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 1, current progress: 1/4.
20/09/10 20:54:49.965 dispatcher-event-loop-13 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:49.966 dispatcher-event-loop-13 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 3, current progress: 2/4.
20/09/10 20:54:50.112 dispatcher-event-loop-16 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:50.113 dispatcher-event-loop-16 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 0, current progress: 3/4.
20/09/10 20:54:50.609 dispatcher-CoarseGrainedScheduler INFO TaskSchedulerImpl: Skip current round of resource offers for barrier stage 0 because the barrier taskSet requires 4 slots, while the total number of available slots is 0.
20/09/10 20:54:50.826 dispatcher-event-loop-17 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:50.827 dispatcher-event-loop-17 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 2, current progress: 4/4.
20/09/10 20:54:50.827 dispatcher-event-loop-17 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received all updates from tasks, finished successfully.
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Updated the test and tested a hundred times without failure(Previously, there could be several failures).
Closes#29732 from Ngone51/fix-flaky-throw-exception.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Increase size of socket receive buffer in these tests.
### Why are the changes needed?
The socket receive buffer size set in this test was too small for
the StatsdSinkSuite tests to run reliably on some systems. For a
test in this suite to run reliably the buffer needs to be large
enough to hold all the data in the packets being sent in a test
along with any additional kernel or protocol overhead. The amount
of kernel overhead per packet can vary from system to system but is
typically far higher than the protocol overhead.
If the receive buffer is too small and fills up then packets are
silently dropped. This leads to the test failing with a timeout.
If the socket defaults to a larger receive buffer (normally true)
then we should keep that size.
As well as increasing the minimum buffer size I've also decoupled
the datagram packet buffer size from the receive buffer size. The
receive buffer should in general be far larger to account for the
fact that multiple packets might be buffered, as well as the
aforementioned overhead. Any truncated data in individual packets
will be picked up by the tests.
### Does this PR introduce _any_ user-facing change?
No, this only affects the tests.
### How was this patch tested?
Existing tests on IBM Z and x86.
Closes#29819 from mundaym/fix-statsd.
Authored-by: Michael Munday <mike.munday@ibm.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Increase the listener bus event length, syncrhonize the addition of blocks modified to the array list.
### Why are the changes needed?
This test appears flaky in Jenkins (can not repro locally). Given that the index file made it through and the index file is only transferred after the data file, the only two reasons I could come up with an interminentent failure here are with the listenerbus dropping a message or the two block change messages being received at the same time.
### Does this PR introduce _any_ user-facing change?
No (test only).
### How was this patch tested?
The tests still pass on my machine but they did before. We'll need to run it through jenkins a few times first.
Closes#29929 from holdenk/fix-.BlockManagerDecommissionIntegrationSuite.
Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Spark has a hardcode list to contain known shuffle managers, which has two values now. It does not contain user's custom shuffle manager which is set through Spark config "spark.shuffle.manager".
We hit issue when set "spark.shuffle.manager" with our own shuffle manager plugin (Uber Remote Shuffle Service implementation, https://github.com/uber/RemoteShuffleService). Other users will hit same issue when they implement their own shuffle manager.
It is better to remove that knownManagers hardcode list, to support user's custom shuffle manager implementation.
### Why are the changes needed?
Spark has shuffle manager API to support custom shuffle manager implementation. The hardcoded known managers list does not consider that shuffle manager config value which could be set by user. Thus better to remove this hardcoded known managers list.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Current Spark unit test already covers the code path.
Closes#29916 from boy-uber/knownManagers.
Lead-authored-by: Bo Yang <boy@uber.com>
Co-authored-by: Bo Yang <boy-uber@users.noreply.github.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
### What changes were proposed in this pull request?
At the moment only the baked in JDBC connection providers can be used but there is a need to support additional databases and use-cases. In this PR I'm proposing a new developer API name `JdbcConnectionProvider`. To show how an external JDBC connection provider can be implemented I've created an example [here](https://github.com/gaborgsomogyi/spark-jdbc-connection-provider).
The PR contains the following changes:
* Added connection provider developer API
* Made JDBC connection providers constructor to noarg => needed to load them w/ service loader
* Connection providers are now loaded w/ service loader
* Added tests to load providers independently
* Moved `SecurityConfigurationLock` into a central place because other areas will change global JVM security config
### Why are the changes needed?
No custom authentication possibility.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
* Existing + additional unit tests
* Docker integration tests
* Tested manually the newly created external JDBC connection provider
Closes#29024 from gaborgsomogyi/SPARK-32001.
Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This change updates the test file location in #29872 to proper path.
### Why are the changes needed?
ExecutorSummarySuite.scala should be in core/src/test/scala instead of core/src/test/java.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit tests
Closes#29926 from shrutig/SPARK-32996.
Authored-by: Shruti Gumma <shruti_gumma@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
### What changes were proposed in this pull request?
Upgrade to the latest available version of jQuery (3.5.1).
### Why are the changes needed?
There are some CVE-s reported (CVE-2020-11022, CVE-2020-11023) affecting older versions of jQuery. Although Spark UI is read-only and those CVEs doesn't seem to affect Spark, using the latest version of this library can help to handle vulnerability reports of security scans.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual tests and checked the jQuery 3.5 upgrade guide.
Closes#29902 from peter-toth/SPARK-32723-upgrade-to-jquery-3.5.1.
Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
according to https://github.com/apache/spark/pull/29881#discussion_r496648397
we need add condition `Utils.isWindows`
### Why are the changes needed?
add strict condition of judging path is window path
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
No
Closes#29909 from AngersZhuuuu/SPARK-33023.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Apache Spark 3.1's default Hadoop profile is `hadoop-3.2`. Instead of having a warning documentation, this PR aims to use a consistent and safer version of Apache Hadoop file output committer algorithm which is `v1`. This will prevent a silent correctness regression during migration from Apache Spark 2.4/3.0 to Apache Spark 3.1.0. Of course, if there is a user-provided configuration, `spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version=2`, that will be used still.
### Why are the changes needed?
Apache Spark provides multiple distributions with Hadoop 2.7 and Hadoop 3.2. `spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version` depends on the Hadoop version. Apache Hadoop 3.0 switches the default algorithm from `v1` to `v2` and now there exists a discussion to remove `v2`. We had better provide a consistent default behavior of `v1` across various Spark distributions.
- [MAPREDUCE-7282](https://issues.apache.org/jira/browse/MAPREDUCE-7282) MR v2 commit algorithm should be deprecated and not the default
### Does this PR introduce _any_ user-facing change?
Yes. This changes the default behavior. Users can override this conf.
### How was this patch tested?
Manual.
**BEFORE (spark-3.0.1-bin-hadoop3.2)**
```scala
scala> sc.version
res0: String = 3.0.1
scala> sc.hadoopConfiguration.get("mapreduce.fileoutputcommitter.algorithm.version")
res1: String = 2
```
**AFTER**
```scala
scala> sc.hadoopConfiguration.get("mapreduce.fileoutputcommitter.algorithm.version")
res0: String = 1
```
Closes#29895 from dongjoon-hyun/SPARK-DEFAUT-COMMITTER.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR changes `UnsafeExternalSorter` to no longer allocate any memory while spilling. In particular it removes the allocation of a new pointer array in `UnsafeInMemorySorter`. Instead the new pointer array is allocated whenever the next record is inserted into the sorter.
### Why are the changes needed?
Without this change the `UnsafeExternalSorter` could throw an OOM while spilling. The following sequence of events would have triggered an OOM:
1. `UnsafeExternalSorter` runs out of space in its pointer array and attempts to allocate a new large array to replace the old one.
2. `TaskMemoryManager` tries to allocate the memory backing the new large array using `MemoryManager`, but `MemoryManager` is only willing to return most but not all of the memory requested.
3. `TaskMemoryManager` asks `UnsafeExternalSorter` to spill, which causes `UnsafeExternalSorter` to spill the current run to disk, to free its record pages and to reset its `UnsafeInMemorySorter`.
4. `UnsafeInMemorySorter` frees the old pointer array, and tries to allocate a new small pointer array.
5. `TaskMemoryManager` tries to allocate the memory backing the small array using `MemoryManager`, but `MemoryManager` is unwilling to give it any memory, as the `TaskMemoryManager` is still holding on to the memory it got for the new large array.
6. `TaskMemoryManager` again asks `UnsafeExternalSorter` to spill, but this time there is nothing to spill.
7. `UnsafeInMemorySorter` receives less memory than it requested, and causes a `SparkOutOfMemoryError` to be thrown, which causes the current task to fail.
With the changes in the PR the following will happen instead:
1. `UnsafeExternalSorter` runs out of space in its pointer array and attempts to allocate a new large array to replace the old one.
2. `TaskMemoryManager` tries to allocate the memory backing the new large array using `MemoryManager`, but `MemoryManager` is only willing to return most but not all of the memory requested.
3. `TaskMemoryManager` asks `UnsafeExternalSorter` to spill, which causes `UnsafeExternalSorter` to spill the current run to disk, to free its record pages and to reset its `UnsafeInMemorySorter`.
4. `UnsafeInMemorySorter` frees the old pointer array.
5. `TaskMemoryManager` returns control to `UnsafeExternalSorter.growPointerArrayIfNecessary` (either by returning the the new large array or by throwing a `SparkOutOfMemoryError`).
6. `UnsafeExternalSorter` either frees the new large array or it ignores the `SparkOutOfMemoryError` depending on what happened in the previous step.
7. `UnsafeExternalSorter` successfully allocates a new small pointer array and operation continues as normal.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Tests were added in `UnsafeExternalSorterSuite` and `UnsafeInMemorySorterSuite`.
Closes#29785 from tomvanbussel/SPARK-32901.
Authored-by: Tom van Bussel <tom.vanbussel@databricks.com>
Signed-off-by: herman <herman@databricks.com>
### What changes were proposed in this pull request?
When `peakMemoryMetrics` in `ExecutorSummary` is `Option.empty`, then the `ExecutorMetricsJsonSerializer#serialize` method does not execute the `jsonGenerator.writeObject` method. This causes the json to be generated with `peakMemoryMetrics` key added to the serialized string, but no corresponding value.
This causes an error to be thrown when it is the next key `attributes` turn to be added to the json:
`com.fasterxml.jackson.core.JsonGenerationException: Can not write a field name, expecting a value
`
### Why are the changes needed?
At the start of the Spark job, if `peakMemoryMetrics` is `Option.empty`, then it causes
a `com.fasterxml.jackson.core.JsonGenerationException` to be thrown when we navigate to the Executors tab in Spark UI.
Complete stacktrace:
> com.fasterxml.jackson.core.JsonGenerationException: Can not write a field name, expecting a value
> at com.fasterxml.jackson.core.JsonGenerator._reportError(JsonGenerator.java:2080)
> at com.fasterxml.jackson.core.json.WriterBasedJsonGenerator.writeFieldName(WriterBasedJsonGenerator.java:161)
> at com.fasterxml.jackson.databind.ser.BeanPropertyWriter.serializeAsField(BeanPropertyWriter.java:725)
> at com.fasterxml.jackson.databind.ser.std.BeanSerializerBase.serializeFields(BeanSerializerBase.java:721)
> at com.fasterxml.jackson.databind.ser.BeanSerializer.serialize(BeanSerializer.java:166)
> at com.fasterxml.jackson.databind.ser.std.CollectionSerializer.serializeContents(CollectionSerializer.java:145)
> at com.fasterxml.jackson.module.scala.ser.IterableSerializer.serializeContents(IterableSerializerModule.scala:26)
> at com.fasterxml.jackson.module.scala.ser.IterableSerializer.serializeContents$(IterableSerializerModule.scala:25)
> at com.fasterxml.jackson.module.scala.ser.UnresolvedIterableSerializer.serializeContents(IterableSerializerModule.scala:54)
> at com.fasterxml.jackson.module.scala.ser.UnresolvedIterableSerializer.serializeContents(IterableSerializerModule.scala:54)
> at com.fasterxml.jackson.databind.ser.std.AsArraySerializerBase.serialize(AsArraySerializerBase.java:250)
> at com.fasterxml.jackson.databind.ser.DefaultSerializerProvider._serialize(DefaultSerializerProvider.java:480)
> at com.fasterxml.jackson.databind.ser.DefaultSerializerProvider.serializeValue(DefaultSerializerProvider.java:319)
> at com.fasterxml.jackson.databind.ObjectMapper._configAndWriteValue(ObjectMapper.java:4094)
> at com.fasterxml.jackson.databind.ObjectMapper.writeValueAsString(ObjectMapper.java:3404)
> at org.apache.spark.ui.exec.ExecutorsPage.allExecutorsDataScript$1(ExecutorsTab.scala:64)
> at org.apache.spark.ui.exec.ExecutorsPage.render(ExecutorsTab.scala:76)
> at org.apache.spark.ui.WebUI.$anonfun$attachPage$1(WebUI.scala:89)
> at org.apache.spark.ui.JettyUtils$$anon$1.doGet(JettyUtils.scala:80)
> at javax.servlet.http.HttpServlet.service(HttpServlet.java:687)
> at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
> at org.sparkproject.jetty.servlet.ServletHolder.handle(ServletHolder.java:873)
> at org.sparkproject.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1623)
> at org.apache.spark.ui.HttpSecurityFilter.doFilter(HttpSecurityFilter.scala:95)
> at org.sparkproject.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1610)
> at org.sparkproject.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:540)
> at org.sparkproject.jetty.server.handler.ScopedHandler.nextHandle(ScopedHandler.java:255)
> at org.sparkproject.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1345)
> at org.sparkproject.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:203)
> at org.sparkproject.jetty.servlet.ServletHandler.doScope(ServletHandler.java:480)
> at org.sparkproject.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:201)
> at org.sparkproject.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1247)
> at org.sparkproject.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:144)
> at org.sparkproject.jetty.server.handler.gzip.GzipHandler.handle(GzipHandler.java:753)
> at org.sparkproject.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:220)
> at org.sparkproject.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:132)
> at org.sparkproject.jetty.server.Server.handle(Server.java:505)
> at org.sparkproject.jetty.server.HttpChannel.handle(HttpChannel.java:370)
> at org.sparkproject.jetty.server.HttpConnection.onFillable(HttpConnection.java:267)
> at org.sparkproject.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:305)
> at org.sparkproject.jetty.io.FillInterest.fillable(FillInterest.java:103)
> at org.sparkproject.jetty.io.ChannelEndPoint$2.run(ChannelEndPoint.java:117)
> at org.sparkproject.jetty.util.thread.strategy.EatWhatYouKill.runTask(EatWhatYouKill.java:333)
> at org.sparkproject.jetty.util.thread.strategy.EatWhatYouKill.doProduce(EatWhatYouKill.java:310)
> at org.sparkproject.jetty.util.thread.strategy.EatWhatYouKill.tryProduce(EatWhatYouKill.java:168)
> at org.sparkproject.jetty.util.thread.strategy.EatWhatYouKill.run(EatWhatYouKill.java:126)
> at org.sparkproject.jetty.util.thread.ReservedThreadExecutor$ReservedThread.run(ReservedThreadExecutor.java:366)
> at org.sparkproject.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:698)
> at org.sparkproject.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:804)
> at java.base/java.lang.Thread.run(Thread.java:834)
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit test
Closes#29872 from shrutig/SPARK-32996.
Authored-by: Shruti Gumma <shruti_gumma@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
<!--
Thanks for sending a pull request! Here are some tips for you:
1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
4. Be sure to keep the PR description updated to reflect all changes.
5. Please write your PR title to summarize what this PR proposes.
6. If possible, provide a concise example to reproduce the issue for a faster review.
7. If you want to add a new configuration, please read the guideline first for naming configurations in
'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
-->
### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
2. If you fix some SQL features, you can provide some references of other DBMSes.
3. If there is design documentation, please add the link.
4. If there is a discussion in the mailing list, please add the link.
-->
This moves and refactors the parallel listing utilities from `InMemoryFileIndex` to Spark core so it can be reused by modules beside SQL. Along the process this also did some cleanups/refactorings:
- Created a `HadoopFSUtils` class under core
- Moved `InMemoryFileIndex.bulkListLeafFiles` into `HadoopFSUtils.parallelListLeafFiles`. It now depends on a `SparkContext` instead of `SparkSession` in SQL. Also added a few parameters which used to be read from `SparkSession.conf`: `ignoreMissingFiles`, `ignoreLocality`, `parallelismThreshold`, `parallelismMax ` and `filterFun` (for additional filtering support but we may be able to merge this with `filter` parameter in future).
- Moved `InMemoryFileIndex.listLeafFiles` into `HadoopFSUtils.listLeafFiles` with similar changes above.
### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
1. If you propose a new API, clarify the use case for a new API.
2. If you fix a bug, you can clarify why it is a bug.
-->
Currently the locality-aware parallel listing mechanism only applies to `InMemoryFileIndex`. By moving this to core, we can potentially reuse the same mechanism for other code paths as well.
### Does this PR introduce _any_ user-facing change?
<!--
Note that it means *any* user-facing change including all aspects such as the documentation fix.
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
If no, write 'No'.
-->
No.
### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
Since this is mostly a refactoring, it relies on existing unit tests such as those for `InMemoryFileIndex`.
Closes#29471 from sunchao/SPARK-32381.
Lead-authored-by: Chao Sun <sunchao@apache.org>
Co-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Chao Sun <sunchao@uber.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
### What changes were proposed in this pull request?
Fixes the log strings the decom integration tests looks for and add comments reminding people to run the K8s integration tests when changing those code paths.
### Why are the changes needed?
The strings it looks for have been changed.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
WIP: Verify that the K8s jenkins job succeeds
Closes#29854 from holdenk/SPARK-32979-spark-k8s-decom-test-is-broken.
Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Fix ".../jobs/undefined" link from "Event Timeline" in jobs page. Job page link in "Event Timeline" view is constructed by fetching job page link defined in job list below. when job count exceeds page size of job table, only links of jobs in job table can be fetched from page. Other jobs' link would be 'undefined', and links of them in "Event Timeline" are broken, they are redirected to some wired URL like ".../jobs/undefined". This PR is fixing this wrong link issue. With this PR, job link in "Event Timeline" view would always redirect to correct job page.
### Why are the changes needed?
Wrong link (".../jobs/undefined") in "Event Timeline" of jobs page. for example, the first job in below page is not in table below, as job count(116) exceeds page size(100). When clicking it's item in "Event Timeline", page is redirected to ".../jobs/undefined", which is wrong. Links in "Event Timeline" should always be correct.
![undefinedlink](https://user-images.githubusercontent.com/10524738/93184779-83fa6d80-f6f1-11ea-8a80-1a304ca9cbb2.JPG)
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manually tested.
Closes#29757 from zhli1142015/fix-link-event-timeline-view.
Authored-by: Zhen Li <zhli@microsoft.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Only calculate the executorRunTime when taskStartTimeNs > 0. Otherwise, set executorRunTime to 0.
### Why are the changes needed?
bug fix.
It's possible that a task be killed (e.g., by another successful attempt) before it reaches "taskStartTimeNs = System.nanoTime()". In this case, taskStartTimeNs is still 0 since it hasn't been really initialized. And we will get the wrong executorRunTime by calculating System.nanoTime() - taskStartTimeNs.
### Does this PR introduce _any_ user-facing change?
Yes, users will see the correct executorRunTime.
### How was this patch tested?
Pass existing tests.
Closes#29789 from Ngone51/fix-SPARK-32898.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR changes `UnsafeExternalSorter.SpillableIterator` to free its memory (except for the page holding the last record) if it is forced to spill after all of its records have been read. It also makes sure that `lastPage` is freed if `loadNext` is never called the again. The latter was necessary to get my test case to succeed (otherwise it would complain about a leak).
### Why are the changes needed?
No memory is freed after calling `UnsafeExternalSorter.SpillableIterator.spill()` when all records have been read, even though it is still holding onto some memory. This may cause a `SparkOutOfMemoryError` to be thrown, even though we could have just freed the memory instead.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
A test was added to `UnsafeExternalSorterSuite`.
Closes#29787 from tomvanbussel/SPARK-32911.
Authored-by: Tom van Bussel <tom.vanbussel@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR aims to replace deprecated `isFile` and `isDirectory` methods.
```diff
- fs.isDirectory(hadoopPath)
+ fs.getFileStatus(hadoopPath).isDirectory
```
```diff
- fs.isFile(new Path(inProgressLog))
+ fs.getFileStatus(new Path(inProgressLog)).isFile
```
### Why are the changes needed?
It shows deprecation warnings.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-3.2-hive-2.3/1244/consoleFull
```
[warn] /home/jenkins/workspace/spark-master-test-sbt-hadoop-3.2-hive-2.3/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala:815: method isFile in class FileSystem is deprecated: see corresponding Javadoc for more information.
[warn] if (!fs.isFile(new Path(inProgressLog))) {
```
```
[warn] /home/jenkins/workspace/spark-master-test-sbt-hadoop-3.2-hive-2.3/core/src/main/scala/org/apache/spark/SparkContext.scala:1884: method isDirectory in class FileSystem is deprecated: see corresponding Javadoc for more information.
[warn] if (fs.isDirectory(hadoopPath)) {
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the Jenkins.
Closes#29796 from williamhyun/filesystem.
Authored-by: William Hyun <williamhyun3@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
To fix the flaky `ExecutorAllocationManagerSuite`: Avoid first `schedule()` invocation after `ExecutorAllocationManager` started.
### Why are the changes needed?
`ExecutorAllocationManagerSuite` is still flaky, see:
https://github.com/apache/spark/pull/29722/checks?check_run_id=1117979237
By checking the below logs, we can see that there's a race condition between thread `pool-1-thread-1-ScalaTest-running` and thread `spark-dynamic-executor-allocation`. The only possibility of thread `spark-dynamic-executor-allocation` becoming active is the first time invocation of `schedule()`(since the `TEST_SCHEDULE_INTERVAL`(30s) is really long, so it's impossible the second invocation would happen). Thus, I think we shall avoid the first invocation too.
```scala
20/09/15 12:41:20.831 pool-1-thread-1-ScalaTest-running-ExecutorAllocationManagerSuite INFO ExecutorAllocationManager: Requesting 1 new executor because tasks are backlogged (new desired total will be 2 for resource profile id: 0)
20/09/15 12:41:20.832 spark-dynamic-executor-allocation INFO ExecutorAllocationManager: Requesting 2 new executors because tasks are backlogged (new desired total will be 4 for resource profile id: 0)
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
The flaky can't be reproduced locally so it's hard to say it has been completely fixed by now. We need time to see the result.
Closes#29773 from Ngone51/fix-SPARK-32287.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR changes the way `UnsafeExternalSorter.SpillableIterator` checks whether it has spilled already, by checking whether `inMemSorter` is null. It also allows it to spill other `UnsafeSorterIterator`s than `UnsafeInMemorySorter.SortedIterator`.
### Why are the changes needed?
Before this PR `UnsafeExternalSorter.SpillableIterator` could not spill when there are NULLs in the input and radix sorting is used. Currently, Spark determines whether UnsafeExternalSorter.SpillableIterator has not spilled yet by checking whether `upstream` is an instance of `UnsafeInMemorySorter.SortedIterator`. When radix sorting is used and there are NULLs in the input however, `upstream` will be an instance of `UnsafeExternalSorter.ChainedIterator` instead, and Spark will assume that the `SpillableIterator` iterator has spilled already, and therefore cannot spill again when it's supposed to spill.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
A test was added to `UnsafeExternalSorterSuite` (and therefore also to `UnsafeExternalSorterRadixSortSuite`). I manually confirmed that the test failed in `UnsafeExternalSorterRadixSortSuite` without this patch.
Closes#29772 from tomvanbussel/SPARK-32900.
Authored-by: Tom van Bussel <tom.vanbussel@databricks.com>
Signed-off-by: herman <herman@databricks.com>
### What changes were proposed in this pull request?
This PR cleans up the RPC message flow among the multiple decommission use cases, it includes changes:
* Keep `Worker`'s decommission status be consistent between the case where decommission starts from `Worker` and the case where decommission starts from the `MasterWebUI`: sending `DecommissionWorker` from `Master` to `Worker` in the latter case.
* Change from two-way communication to one-way communication when notifying decommission between driver and executor: it's obviously unnecessary for the executor to acknowledge the decommission status to the driver since the decommission request is from the driver. And it's same in reverse.
* Only send one message instead of two(`DecommissionSelf`/`DecommissionBlockManager`) when decommission the executor: executor and `BlockManager` are in the same JVM.
* Clean up codes around here.
### Why are the changes needed?
Before:
<img width="1948" alt="WeChat56c00cc34d9785a67a544dca036d49da" src="https://user-images.githubusercontent.com/16397174/92850308-dc461c80-f41e-11ea-8ac0-287825f4e0c4.png">
After:
<img width="1968" alt="WeChat05f7afb017e3f0132394c5e54245e49e" src="https://user-images.githubusercontent.com/16397174/93189571-de88dd80-f774-11ea-9300-1943920aa27d.png">
(Note the diagrams only counts those RPC calls that needed to go through the network. Local RPC calls are not counted here.)
After this change, We reduced 6 original RPC calls and added one more RPC call for keeping the consistent decommission status for the Worker. And the RPC flow becomes more clear.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Updated existing tests.
Closes#29722 from Ngone51/simplify-decommission-rpc.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Processing for `ThreadSafeRpcEndpoint` is controlled by `numActiveThreads` in `Inbox`. Now if any fatal error happens during `Inbox.process`, `numActiveThreads` is not reduced. Then other threads can not process messages in that inbox, which causes the endpoint to "hang". For other type of endpoints, we also should keep `numActiveThreads` correct.
This problem is more serious in previous Spark 2.x versions since the driver, executor and block manager endpoints are all thread safe endpoints.
To fix this, we should reduce the number of active threads if fatal error happens in `Inbox.process`.
### Why are the changes needed?
`numActiveThreads` is not correct when fatal error happens and will cause the described problem.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Add a new test.
Closes#29580 from wzhfy/deal_with_fatal_error.
Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR proposes to avoid scheduling the (non-zombie) TaskSetManager which has no pending tasks.
### Why are the changes needed?
Currently, Spark always tries to schedule a (non-zombie) TaskSetManager even if it has no pending tasks. This causes notable problems for the barrier TaskSetManager: 1. `calculateAvailableSlots` can be called for multiple times for a launched barrier TaskSetManager; 2. user would see "Skip current round of resource offers for barrier stage" log message for
a launched barrier TaskSetManager all the time until the barrier TaskSetManager finishes, which is quite confused.
Besides, scheduling a TaskSetManager always involves many function invocations even if there're no pending tasks.
Therefore, I think we can skip those un-schedulable TasksetManagers to avoid the potential overhead.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass existing tests.
Closes#29750 from Ngone51/filter-out-unschedulable-stage.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
In TorrentBroadcast.scala
```scala
L133: if (!blockManager.putSingle(broadcastId, value, MEMORY_AND_DISK, tellMaster = false))
L137: TorrentBroadcast.blockifyObject(value, blockSize, SparkEnv.get.serializer, compressionCodec)
L147: if (!blockManager.putBytes(pieceId, bytes, MEMORY_AND_DISK_SER, tellMaster = true))
```
After the original value is saved successfully(TorrentBroadcast.scala: L133), but the following `blockifyObject()`(L137) or store piece(L147) steps are failed. There is no opportunity to release broadcast from memory.
This patch is to remove all pieces of the broadcast when failed to blockify or failed to store some pieces of a broadcast.
### Why are the changes needed?
We use Spark thrift-server as a long-running service. A bad query submitted a heavy BroadcastNestLoopJoin operation and made driver full GC. We killed the bad query but we found the driver's memory usage was still high and full GCs were still frequent. By investigating with GC dump and log, we found the broadcast may memory leak.
> 2020-08-19T18:54:02.824-0700: [Full GC (Allocation Failure)
2020-08-19T18:54:02.824-0700: [Class Histogram (before full gc):
116G->112G(170G), 184.9121920 secs]
[Eden: 32.0M(7616.0M)->0.0B(8704.0M) Survivors: 1088.0M->0.0B Heap: 116.4G(170.0G)->112.9G(170.0G)], [Metaspace: 177285K->177270K(182272K)]
1: 676531691 72035438432 [B
2: 676502528 32472121344 org.apache.spark.sql.catalyst.expressions.UnsafeRow
3: 99551 12018117568 [Ljava.lang.Object;
4: 26570 4349629040 [I
5: 6 3264536688 [Lorg.apache.spark.sql.catalyst.InternalRow;
6: 1708819 256299456 [C
7: 2338 179615208 [J
8: 1703669 54517408 java.lang.String
9: 103860 34896960 org.apache.spark.status.TaskDataWrapper
10: 177396 25545024 java.net.URI
...
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manually test. This UT is hard to write and the patch is straightforward.
Closes#29558 from LantaoJin/SPARK-32715.
Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
When BytesToBytesMap is at `MAX_CAPACITY` and reaches its growth threshold, `numKeys >= growthThreshold` is true but `longArray.size() / 2 < MAX_CAPACITY` is false. This correctly prevents the map from growing, but `canGrowArray` incorrectly remains true. Therefore the map keeps accepting new keys and exceeds its growth threshold. If we attempt to spill the map in this state, the UnsafeKVExternalSorter will not be able to reuse the long array for sorting. By this point the task has typically consumed all available memory, so the allocation of the new pointer array is likely to fail.
This PR fixes the issue by setting `canGrowArray` to false in this case. This prevents the map from accepting new elements when it cannot grow to accommodate them.
### Why are the changes needed?
Without this change, hash aggregations will fail when the number of groups per task is greater than `MAX_CAPACITY / 2 = 2^28` (approximately 268 million), and when the grouping aggregation is the only memory-consuming operator in its stage.
For example, the final aggregation in `SELECT COUNT(DISTINCT id) FROM tbl` fails when `tbl` contains 1 billion distinct values and when `spark.sql.shuffle.partitions=1`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Reproducing this issue requires building a very large BytesToBytesMap. Because this is infeasible to do in a unit test, this PR was tested manually by adding the following test to AbstractBytesToBytesMapSuite. Before this PR, the test fails in 8.5 minutes. With this PR, the test passes in 1.5 minutes.
```java
public abstract class AbstractBytesToBytesMapSuite {
// ...
Test
public void respectGrowthThresholdAtMaxCapacity() {
TestMemoryManager memoryManager2 =
new TestMemoryManager(
new SparkConf()
.set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), true)
.set(package$.MODULE$.MEMORY_OFFHEAP_SIZE(), 25600 * 1024 * 1024L)
.set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false)
.set(package$.MODULE$.SHUFFLE_COMPRESS(), false));
TaskMemoryManager taskMemoryManager2 = new TaskMemoryManager(memoryManager2, 0);
final long pageSizeBytes = 8000000 + 8; // 8 bytes for end-of-page marker
final BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager2, 1024, pageSizeBytes);
try {
// Insert keys into the map until it stops accepting new keys.
for (long i = 0; i < BytesToBytesMap.MAX_CAPACITY; i++) {
if (i % (1024 * 1024) == 0) System.out.println("Inserting element " + i);
final long[] value = new long[]{i};
BytesToBytesMap.Location loc = map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8);
Assert.assertFalse(loc.isDefined());
boolean success =
loc.append(value, Platform.LONG_ARRAY_OFFSET, 8, value, Platform.LONG_ARRAY_OFFSET, 8);
if (!success) break;
}
// The map should grow to its max capacity.
long capacity = map.getArray().size() / 2;
Assert.assertTrue(capacity == BytesToBytesMap.MAX_CAPACITY);
// The map should stop accepting new keys once it has reached its growth
// threshold, which is half the max capacity.
Assert.assertTrue(map.numKeys() == BytesToBytesMap.MAX_CAPACITY / 2);
map.free();
} finally {
map.free();
}
}
}
```
Closes#29744 from ankurdave/SPARK-32872.
Authored-by: Ankur Dave <ankurdave@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
`DAGSchedulerSuite` exists some issue:
`afterEach` and `init` are called when the `SparkConf` of the default `SparkContext` has no configuration that the test case must set. This causes the `SparkContext` initialized in `beforeEach` to be discarded without being used, resulting in waste. On the other hand, the flexibility to add configurations to `SparkConf` should be addressed by the test framework.
Test suites inherits `LocalSparkContext` can be simplified.
### Why are the changes needed?
Reduce overhead about init `SparkContext`.
Rewrite the test framework to support apply specified spark configurations.
### Does this PR introduce _any_ user-facing change?
'No'.
### How was this patch tested?
Jenkins test.
Closes#29228 from beliefer/extend-test-frame-for-dag.
Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
The purpose of this pr is to partial resolve [SPARK-32808](https://issues.apache.org/jira/browse/SPARK-32808), total of 26 failed test cases were fixed, the related suite as follow:
- `StreamingAggregationSuite` related test cases (2 FAILED -> Pass)
- `GeneratorFunctionSuite` related test cases (2 FAILED -> Pass)
- `UDFSuite` related test cases (2 FAILED -> Pass)
- `SQLQueryTestSuite` related test cases (5 FAILED -> Pass)
- `WholeStageCodegenSuite` related test cases (1 FAILED -> Pass)
- `DataFrameSuite` related test cases (3 FAILED -> Pass)
- `OrcV1QuerySuite\OrcV2QuerySuite` related test cases (4 FAILED -> Pass)
- `ExpressionsSchemaSuite` related test cases (1 FAILED -> Pass)
- `DataFrameStatSuite` related test cases (1 FAILED -> Pass)
- `JsonV1Suite\JsonV2Suite\JsonLegacyTimeParserSuite` related test cases (6 FAILED -> Pass)
The main change of this pr as following:
- Fix Scala 2.13 compilation problems in `ShuffleBlockFetcherIterator` and `Analyzer`
- Specified `Seq` to `scala.collection.Seq` in `objects.scala` and `GenericArrayData` because internal use `Seq` maybe `mutable.ArraySeq` and not easy to call `.toSeq`
- Should specified `Seq` to `scala.collection.Seq` when we call `Row.getAs[Seq]` and `Row.get(i).asInstanceOf[Seq]` because the data maybe `mutable.ArraySeq` but `Seq` is `immutable.Seq` in Scala 2.13
- Use a compatible way to let `+` and `-` method of `Decimal` having the same behavior in Scala 2.12 and Scala 2.13
- Call `toList` in `RelationalGroupedDataset.toDF` method when `groupingExprs` is `Stream` type because `Stream` can't serialize in Scala 2.13
- Add a manual sort to `classFunsMap` in `ExpressionsSchemaSuite` because `Iterable.groupBy` in Scala 2.13 has different result with `TraversableLike.groupBy` in Scala 2.12
### Why are the changes needed?
We need to support a Scala 2.13 build.
### Does this PR introduce _any_ user-facing change?
Should specified `Seq` to `scala.collection.Seq` when we call `Row.getAs[Seq]` and `Row.get(i).asInstanceOf[Seq]` because the data maybe `mutable.ArraySeq` but the `Seq` is `immutable.Seq` in Scala 2.13
### How was this patch tested?
- Scala 2.12: Pass the Jenkins or GitHub Action
- Scala 2.13: Do the following:
```
dev/change-scala-version.sh 2.13
mvn clean install -DskipTests -pl sql/core -Pscala-2.13 -am
mvn test -pl sql/core -Pscala-2.13
```
**Before**
```
Tests: succeeded 8166, failed 319, canceled 1, ignored 52, pending 0
*** 319 TESTS FAILED ***
```
**After**
```
Tests: succeeded 8204, failed 286, canceled 1, ignored 52, pending 0
*** 286 TESTS FAILED ***
```
Closes#29660 from LuciferYang/SPARK-32808.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Fixes the master UI for properly summing the resources total across multiple workers.
field:
Resources in use: 0 / 8 gpu
The bug here is that it was creating MutableResourceInfo and then reducing using the + operator. the + operator in MutableResourceInfo simple adds the address from one to the addresses of the other. But its using a HashSet so if the addresses are the same then you lose the correct amount. ie worker1 has gpu addresses 0,1,2,3 and worker2 has addresses 0,1,2,3 then you only see 4 total GPUs when there are 8.
In this case we don't really need to create the MutableResourceInfo at all because we just want the sums for used and total so just remove the use of it. The other uses of it are per Worker so those should be ok.
### Why are the changes needed?
fix UI
### Does this PR introduce _any_ user-facing change?
UI
### How was this patch tested?
tested manually on standalone cluster with multiple workers and multiple GPUs and multiple fpgas
Closes#29683 from tgravescs/SPARK-32823.
Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
If the user forgets to specify .amount on a resource config like spark.executor.resource.gpu, the error message thrown is very confusing:
```
ERROR SparkContext: Error initializing SparkContext.java.lang.StringIndexOutOfBoundsException: String index out of range:
-1 at java.lang.String.substring(String.java:1967) at
```
This makes it so we have a readable error thrown
### Why are the changes needed?
confusing error for users
### Does this PR introduce _any_ user-facing change?
just error message
### How was this patch tested?
Tested manually on standalone cluster
Closes#29685 from tgravescs/SPARK-32824.
Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
The motivation of this PR is to avoid caching the removed decommissioned executors in `TaskSchedulerImpl`. The cache is introduced in https://github.com/apache/spark/pull/29422. The cache will hold the `isHostDecommissioned` info for a while. So if the task `FetchFailure` event comes after the executor loss event, `DAGScheduler` can still get the `isHostDecommissioned` from the cache and unregister the host shuffle map status when the host is decommissioned too.
This PR tries to achieve the same goal without the cache. Instead of saving the `workerLost` in `ExecutorUpdated` / `ExecutorDecommissionInfo` / `ExecutorDecommissionState`, we could save the `hostOpt` directly. When the host is decommissioned or lost too, the `hostOpt` can be a specific host address. Otherwise, it's `None` to indicate that only the executor is decommissioned or lost.
Now that we have the host info, we can also unregister the host shuffle map status when `executorLost` is triggered for the decommissioned executor.
Besides, this PR also includes a few cleanups around the touched code.
### Why are the changes needed?
It helps to unregister the shuffle map status earlier for both decommission and normal executor lost cases.
It also saves memory in `TaskSchedulerImpl` and simplifies the code a little bit.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
This PR only refactor the code. The original behaviour should be covered by `DecommissionWorkerSuite`.
Closes#29579 from Ngone51/impr-decom.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR adds support to read host-local shuffle data from disk directly when external shuffle service is disabled.
Similar to #25299, we first try to get local disk directories for the shuffle data, which is located at the same host with the current executor. The only difference is, in #25299, it gets the directories from the external shuffle service while in this PR, it gets the directory from the executors.
To implement the feature, this PR extends the `HostLocalDirManager ` for both `ExternalBlockStoreClient` and `NettyBlockTransferService`. Also, this PR adds `getHostLocalDirs` for `NettyBlockTransferService` as `ExternalBlockStoreClient` does, in order to send the get-dir-request to the corresponding executor. And this PR resued the request message`GetLocalDirsForExecutors` for simple.
### Why are the changes needed?
After SPARK-27651 / #25299, Spark can read host-local shuffle data directly from disk when external shuffle service is enabled. To extend the future, we can also support it when the external shuffle service is disabled.
### Does this PR introduce _any_ user-facing change?
Yes. Before this PR, to use the host-local shuffle reading feature, users should not only enable `spark.shuffle.readHostLocalDisk` but also `spark.shuffle.service.enabled`. After this PR, enable `spark.shuffle.readHostLocalDisk` should be enough, and external shuffle service is no longer a pre-requirement.
### How was this patch tested?
Added test and tested manually.
Closes#28911 from Ngone51/support_node_local_shuffle.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
#29401 move `test_script.py` from sql/hive module to sql/core module, cause HiveScripTransformationSuite load resource issue.
### Why are the changes needed?
This issue cause jenkins test failed in mvn
spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11: https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/
spark-master-test-maven-hadoop-3.2-hive-2.3-jdk-11:
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-hive-2.3-jdk-11/
spark-master-test-maven-hadoop-3.2-hive-2.3:
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-hive-2.3/
![image](https://user-images.githubusercontent.com/46485123/91681585-71285a80-eb81-11ea-8519-99fc9783d6b9.png)
![image](https://user-images.githubusercontent.com/46485123/91681010-aaf86180-eb7f-11ea-8dbb-61365a3b0ab4.png)
Error as below:
```
Exception thrown while executing Spark plan:
HiveScriptTransformation [a#349299, b#349300, c#349301, d#349302, e#349303], python /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/hive/file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar!/test_script.py, [a#349309, b#349310, c#349311, d#349312, e#349313], ScriptTransformationIOSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),List((field.delim, )),List((field.delim, )),Some(org.apache.hadoop.hive.ql.exec.TextRecordReader),Some(org.apache.hadoop.hive.ql.exec.TextRecordWriter),false)
+- Project [_1#349288 AS a#349299, _2#349289 AS b#349300, _3#349290 AS c#349301, _4#349291 AS d#349302, _5#349292 AS e#349303]
+- LocalTableScan [_1#349288, _2#349289, _3#349290, _4#349291, _5#349292]
== Exception ==
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 18021.0 failed 1 times, most recent failure: Lost task 0.0 in stage 18021.0 (TID 37324) (192.168.10.31 executor driver): org.apache.spark.SparkException: Subprocess exited with status 2. Error: python: can't open file '/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/hive/file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar!/test_script.py': [Errno 2] No such file or directory
at org.apache.spark.sql.execution.BaseScriptTransformationExec.checkFailureAndPropagate(BaseScriptTransformationExec.scala:180)
at org.apache.spark.sql.execution.BaseScriptTransformationExec.checkFailureAndPropagate$(BaseScriptTransformationExec.scala:157)
at org.apache.spark.sql.hive.execution.HiveScriptTransformationExec.checkFailureAndPropagate(HiveScriptTransformationExec.scala:49)
at org.apache.spark.sql.hive.execution.HiveScriptTransformationExec$$anon$1.hasNext(HiveScriptTransformationExec.scala:110)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
at org.apache.spark.scheduler.Task.run(Task.scala:127)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:480)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1426)
at o
```
### Does this PR introduce _any_ user-facing change?
NO
### How was this patch tested?
Existed UT
Closes#29588 from AngersZhuuuu/SPARK-32400-FOLLOWUP.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Instead of deleting the data, we can move the data to trash.
Based on the configuration provided by the user it will be deleted permanently from the trash.
### Why are the changes needed?
Instead of directly deleting the data, we can provide flexibility to move data to the trash and then delete it permanently.
### Does this PR introduce _any_ user-facing change?
Yes, After truncate table the data is not permanently deleted now.
It is first moved to the trash and then after the given time deleted permanently;
### How was this patch tested?
new UTs added
Closes#29552 from Udbhav30/truncate.
Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
The decommissioning state is a bit fragment across two places in the TaskSchedulerImpl:
https://github.com/apache/spark/pull/29014/ stored the incoming decommission info messages in TaskSchedulerImpl.executorsPendingDecommission.
While https://github.com/apache/spark/pull/28619/ was storing just the executor end time in the map TaskSetManager.tidToExecutorKillTimeMapping (which in turn is contained in TaskSchedulerImpl).
While the two states are not really overlapping, it's a bit of a code hygiene concern to save this state in two places.
With https://github.com/apache/spark/pull/29422, TaskSchedulerImpl is emerging as the place where all decommissioning book keeping is kept within the driver. So consolidate the information in _tidToExecutorKillTimeMapping_ into _executorsPendingDecommission_.
However, in order to do so, we need to walk away from keeping the raw ExecutorDecommissionInfo messages and instead keep another class ExecutorDecommissionState. This decoupling will allow the RPC message class ExecutorDecommissionInfo to evolve independently from the book keeping ExecutorDecommissionState.
### Why are the changes needed?
This is just a code cleanup. These two features were added independently and its time to consolidate their state for good hygiene.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Existing tests.
Closes#29452 from agrawaldevesh/consolidate_decom_state.
Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
### What changes were proposed in this pull request?
Instead of deleting the data, we can move the data to trash.
Based on the configuration provided by the user it will be deleted permanently from the trash.
### Why are the changes needed?
Instead of directly deleting the data, we can provide flexibility to move data to the trash and then delete it permanently.
### Does this PR introduce _any_ user-facing change?
Yes, After truncate table the data is not permanently deleted now.
It is first moved to the trash and then after the given time deleted permanently;
### How was this patch tested?
new UTs added
Closes#29387 from Udbhav30/tuncateTrash.
Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
fixing flaky test in ExecutorAllocationManagerSuite. The issue is that there is a timing issue when we do a reset as to when the numExecutorsToAddPerResourceProfileId gets reset. The fix is to just always set those values back to 1 when we call reset().
### Why are the changes needed?
fixing flaky test in ExecutorAllocationManagerSuite
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
ran the unit test via this PR a bunch of times and the fix seems to be working.
Closes#29508 from tgravescs/debugExecAllocTest.
Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Changing an info log to a debug log based on SPARK-32664
### Why are the changes needed?
It is outlined in SPARK-32664
### Does this PR introduce _any_ user-facing change?
There are changes to the debug and info logs
### How was this patch tested?
Tested by looking at the logs
Closes#29527 from dmoore62/SPARK-32664.
Authored-by: Daniel Moore <moore@knights.ucf.edu>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This pull request adds 2 test suites for 2 new classes HybridStore and HistoryServerMemoryManager, which were created in https://github.com/apache/spark/pull/28412. This pull request also did some minor changes in these 2 classes to expose some variables for testing. Besides 2 suites, this pull request adds a unit test in FsHistoryProviderSuite to test parsing logs with HybridStore.
### Why are the changes needed?
Unit tests are needed for new features.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Unit tests.
Closes#29509 from baohe-zhang/SPARK-31608-UT.
Authored-by: Baohe Zhang <baohe.zhang@verizonmedia.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
In order to produce consistent results from SizeEstimator the tests
override some system properties that are used during SizeEstimator
initialization. However there were several places where either the
compressed references property wasn't set or the system properties
were set but the SizeEstimator not re-initialized.
This caused failures when running the tests with a large heap build
of OpenJ9 because it does not use compressed references unlike most
environments.
### What changes were proposed in this pull request?
Initialize SizeEstimator class explicitly in the tests where required to avoid relying on a particular environment.
### Why are the changes needed?
Test failures can be seen when compressed references are disabled (e.g. using an OpenJ9 large heap build or Hotspot with a large heap).
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Tests run on machine running OpenJ9 large heap build.
Closes#29407 from mundaym/fix-sizeestimator.
Authored-by: Michael Munday <mike.munday@ibm.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
Simple typo
### What changes were proposed in this pull request?
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#29486 from Smith-Cruise/patch-1.
Authored-by: Smith Cruise <chendingchao1@126.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
The purpose of this pr is to resolve [SPARK-32526](https://issues.apache.org/jira/browse/SPARK-32526), all remaining failed cases are fixed.
The main change of this pr as follow:
- Change of `ExecutorAllocationManager.scala` for core module compilation in Scala 2.13, it's a blocking problem
- Change `Seq[_]` to `scala.collection.Seq[_]` refer to failed cases
- Added different expected plan of `Test 4: Star with several branches` of StarJoinCostBasedReorderSuite for Scala 2.13 because the candidates plans:
```
Join Inner, (d1_pk#5 = f1_fk1#0)
:- Join Inner, (f1_fk2#1 = d2_pk#8)
: :- Join Inner, (f1_fk3#2 = d3_pk#11)
```
and
```
Join Inner, (f1_fk2#1 = d2_pk#8)
:- Join Inner, (d1_pk#5 = f1_fk1#0)
: :- Join Inner, (f1_fk3#2 = d3_pk#11)
```
have same cost `Cost(200,9200)`, but `HashMap` is rewritten in scala 2.13 and The order of iterations leads to different results.
This pr fix test cases as follow:
- LiteralExpressionSuite (1 FAILED -> PASS)
- StarJoinCostBasedReorderSuite ( 1 FAILED-> PASS)
- ObjectExpressionsSuite( 2 FAILED-> PASS)
- ScalaReflectionSuite (1 FAILED-> PASS)
- RowEncoderSuite (10 FAILED-> PASS)
- ExpressionEncoderSuite (ABORTED-> PASS)
### Why are the changes needed?
We need to support a Scala 2.13 build.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
<!--
- Scala 2.12: Pass the Jenkins or GitHub Action
- Scala 2.13: Do the following:
```
dev/change-scala-version.sh 2.13
mvn clean install -DskipTests -pl sql/catalyst -Pscala-2.13 -am
mvn test -pl sql/catalyst -Pscala-2.13
```
**Before**
```
Tests: succeeded 4035, failed 17, canceled 0, ignored 6, pending 0
*** 1 SUITE ABORTED ***
*** 15 TESTS FAILED ***
```
**After**
```
Tests: succeeded 4338, failed 0, canceled 0, ignored 6, pending 0
All tests passed.
```
Closes#29434 from LuciferYang/sql-catalyst-tests.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Fix typo for docs, log messages and comments
### Why are the changes needed?
typo fix to increase readability
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
manual test has been performed to test the updated
Closes#29443 from brandonJY/spell-fix-doc.
Authored-by: Brandon Jiang <Brandon.jiang.a@outlook.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Add decommissioning status checking for a host or executor while checking it's active or not. And a decommissioned host or executor should be considered as inactive.
### Why are the changes needed?
First of all, this PR is not a correctness bug fix but gives improvement indeed. And the main problem here we want to fix is that a decommissioned host or executor should be considered as inactive.
`TaskSetManager.computeValidLocalityLevels` depends on `TaskSchedulerImpl.isExecutorAlive/hasExecutorsAliveOnHost` to calculate the locality levels. Therefore, the `TaskSetManager` could also get corresponding locality levels of those decommissioned hosts or executors if they're not considered as inactive. However, on the other side, `CoarseGrainedSchedulerBackend` won't construct the `WorkerOffer` for those decommissioned executors. That also means `TaskSetManager` might never have a chance to launch tasks at certain locality levels but only suffers the unnecessary delay because of delay scheduling. So, this PR helps to reduce this kind of unnecessary delay by making decommissioned host/executor inactive in `TaskSchedulerImpl`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added unit tests
Closes#29468 from Ngone51/fix-decom-alive.
Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Co-authored-by: wuyi <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
The `count` in `PartitionWriterStream` should be a long value, instead of int. The issue is introduced by apache/sparkabef84a . When the overflow happens, the shuffle index file would record wrong index of a reduceId, thus lead to `FetchFailedException: Stream is corrupted` error.
Besides the fix, I also added some debug logs, so in the future it's easier to debug similar issues.
### Why are the changes needed?
This is a regression and bug fix.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
A Spark user reported this issue when migrating their workload to 3.0. One of the jobs fail deterministically on Spark 3.0 without the patch, and the job succeed after applied the fix.
Closes#29474 from jiangxb1987/fixPartitionWriteStream.
Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR replaces some arbitrary task names in logs with the widely used task name (e.g. "task 0.0 in stage 1.0 (TID 1)") among driver and executor. This will change the task name in `TaskDescription` by appending TID.
### Why are the changes needed?
Some logs are still using TID(a.k.a `taskId`) only as the task name, e.g.,
7f275ee597/core/src/main/scala/org/apache/spark/executor/Executor.scala (L786)7f275ee597/core/src/main/scala/org/apache/spark/executor/Executor.scala (L632-L635)
And the task thread name also only has the `taskId`:
7f275ee597/core/src/main/scala/org/apache/spark/executor/Executor.scala (L325)
As mentioned in https://github.com/apache/spark/pull/1259, TID itself does not capture stage or retries, making it harder to correlate with the application. It's inconvenient when debugging applications.
Actually, task name like "task name (e.g. "task 0.0 in stage 1.0 (TID 1)")" has already been used widely after https://github.com/apache/spark/pull/1259. We'd better follow the naming convention.
### Does this PR introduce _any_ user-facing change?
Yes. Users will see the more consistent task names in the log.
### How was this patch tested?
Manually checked.
Closes#29418 from Ngone51/unify-task-name.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Rename `spark.worker.decommission.enabled` to `spark.decommission.enabled` and move it from `org.apache.spark.internal.config.Worker` to `org.apache.spark.internal.config.package`.
### Why are the changes needed?
Decommission has been supported in Standalone and k8s yet and may be supported in Yarn(https://github.com/apache/spark/pull/27636) in the future. Therefore, the switch configuration should have the highest hierarchy rather than belongs to Standalone's Worker. In other words, it should be independent of the cluster managers.
### Does this PR introduce _any_ user-facing change?
No, as the decommission feature hasn't been released.
### How was this patch tested?
Pass existed tests.
Closes#29466 from Ngone51/fix-decom-conf.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Only calculate `executorKillTime` in `TaskSetManager.executorDecommission()` when speculation is enabled.
### Why are the changes needed?
Avoid unnecessary operations to save time/memory.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass existed tests.
Closes#29464 from Ngone51/followup-SPARK-21040.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
The DecommissionWorkerSuite started becoming flaky and it revealed a real regression. Recently closed#29211 necessitates remembering the decommissioning shortly beyond the removal of the executor.
In addition to fixing this issue, ensure that DecommissionWorkerSuite continues to pass when executors haven't had a chance to exit eagery. That is the old behavior before #29211 also still works.
Added some more tests to TaskSchedulerImpl to ensure that the decommissioning information is indeed purged after a timeout.
Hardened the test DecommissionWorkerSuite to make it wait for successful job completion.
### Why are the changes needed?
First, let me describe the intended behavior of decommissioning: If a fetch failure happens where the source executor was decommissioned, we want to treat that as an eager signal to clear all shuffle state associated with that executor. In addition if we know that the host was decommissioned, we want to forget about all map statuses from all other executors on that decommissioned host. This is what the test "decommission workers ensure that fetch failures lead to rerun" is trying to test. This invariant is important to ensure that decommissioning a host does not lead to multiple fetch failures that might fail the job. This fetch failure can happen before the executor is truly marked "lost" because of heartbeat delays.
- However, #29211 eagerly exits the executors when they are done decommissioning. This removal of the executor was racing with the fetch failure. By the time the fetch failure is triggered the executor is already removed and thus has forgotten its decommissioning information. (I tested this by delaying the decommissioning). The fix is to keep the decommissioning information around for some time after removal with some extra logic to finally purge it after a timeout.
- In addition the executor loss can also bump up `shuffleFileLostEpoch` (added in #28848). This happens because when the executor is lost, it forgets the shuffle state about just that executor and increments the `shuffleFileLostEpoch`. This incrementing precludes the clearing of state of the entire host when the fetch failure happens because the failed task is still reusing the old epoch. The fix here is also simple: Ignore the `shuffleFileLostEpoch` when the shuffle status is being cleared due to a fetch failure resulting from host decommission.
I am strategically making both of these fixes be very local to decommissioning to avoid other regressions. Especially the version stuff is tricky (it hasn't been fundamentally changed since it was first introduced in 2013).
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manually ran DecommissionWorkerSuite several times using a script and ensured it all passed.
### (Internal) Configs added
I added two configs, one of which is sort of meant for testing only:
- `spark.test.executor.decommission.initial.sleep.millis`: Initial delay by the decommissioner shutdown thread. Default is same as before of 1 second. This is used for testing only. This one is kept "hidden" (ie not added as a constant to avoid config bloat)
- `spark.executor.decommission.removed.infoCacheTTL`: Number of seconds to keep the removed executors decom entries around. It defaults to 5 minutes. It should be around the average time it takes for all of the shuffle data to be fetched from the mapper to the reducer, but I think that can take a while since the reducers also do a multistep sort.
Closes#29422 from agrawaldevesh/decom_fixes.
Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add support for full outer join inside shuffled hash join. Currently if the query is a full outer join, we only use sort merge join as the physical operator. However it can be CPU and IO intensive in case input table is large for sort merge join. Shuffled hash join on the other hand saves the sort CPU and IO compared to sort merge join, especially when table is large.
This PR implements the full outer join as followed:
* Process rows from stream side by looking up hash relation, and mark the matched rows from build side by:
* for joining with unique key, a `BitSet` is used to record matched rows from build side (`key index` to represent each row)
* for joining with non-unique key, a `HashSet[Long]` is used to record matched rows from build side (`key index` + `value index` to represent each row).
`key index` is defined as the index into key addressing array `longArray` in `BytesToBytesMap`.
`value index` is defined as the iterator index of values for same key.
* Process rows from build side by iterating hash relation, and filter out rows from build side being looked up already (done in `ShuffledHashJoinExec.fullOuterJoin`)
For context, this PR was originally implemented as followed (up to commit e3322766d4):
1. Construct hash relation from build side, with extra boolean value at the end of row to track look up information (done in `ShuffledHashJoinExec.buildHashedRelation` and `UnsafeHashedRelation.apply`).
2. Process rows from stream side by looking up hash relation, and mark the matched rows from build side be looked up (done in `ShuffledHashJoinExec.fullOuterJoin`).
3. Process rows from build side by iterating hash relation, and filter out rows from build side being looked up already (done in `ShuffledHashJoinExec.fullOuterJoin`).
See discussion of pros and cons between these two approaches [here](https://github.com/apache/spark/pull/29342#issuecomment-672275450), [here](https://github.com/apache/spark/pull/29342#issuecomment-672288194) and [here](https://github.com/apache/spark/pull/29342#issuecomment-672640531).
TODO: codegen for full outer shuffled hash join can be implemented in another followup PR.
### Why are the changes needed?
As implementation in this PR, full outer shuffled hash join will have overhead to iterate build side twice (once for building hash map, and another for outputting non-matching rows), and iterate stream side once. However, full outer sort merge join needs to iterate both sides twice, and sort the large table can be more CPU and IO intensive. So full outer shuffled hash join can be more efficient than sort merge join when stream side is much more larger than build side.
For example query below, full outer SHJ saved 30% wall clock time compared to full outer SMJ.
```
def shuffleHashJoin(): Unit = {
val N: Long = 4 << 22
withSQLConf(
SQLConf.SHUFFLE_PARTITIONS.key -> "2",
SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000") {
codegenBenchmark("shuffle hash join", N) {
val df1 = spark.range(N).selectExpr(s"cast(id as string) as k1")
val df2 = spark.range(N / 10).selectExpr(s"cast(id * 10 as string) as k2")
val df = df1.join(df2, col("k1") === col("k2"), "full_outer")
df.noop()
}
}
}
```
```
Running benchmark: shuffle hash join
Running case: shuffle hash join off
Stopped after 2 iterations, 16602 ms
Running case: shuffle hash join on
Stopped after 5 iterations, 31911 ms
Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU 2.40GHz
shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join off 7900 8301 567 2.1 470.9 1.0X
shuffle hash join on 6250 6382 95 2.7 372.5 1.3X
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added unit test in `JoinSuite.scala`, `AbstractBytesToBytesMapSuite.java` and `HashedRelationSuite.scala`.
Closes#29342 from c21/full-outer-shj.
Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
This PR changes Executor to load jars and files added by --jars and --files on Executor initialization.
To avoid downloading those jars/files twice, they are assosiated with `startTime` as their uploaded timestamp.
### Why are the changes needed?
ExecutorPlugin can't work with Standalone Cluster and Kubernetes
when a jar which contains plugins and files used by the plugins are added by --jars and --files option with spark-submit.
This is because jars and files added by --jars and --files are not loaded on Executor initialization.
I confirmed it works with YARN because jars/files are distributed as distributed cache.
### Does this PR introduce _any_ user-facing change?
Yes. jars/files added by --jars and --files are downloaded on each executor on initialization.
### How was this patch tested?
Added a new testcase.
Closes#28939 from sarutak/fix-plugin-issue.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
### What changes were proposed in this pull request?
If graceful decommissioning is enabled, Spark's dynamic scaling uses this instead of directly killing executors.
### Why are the changes needed?
When scaling down Spark we should avoid triggering recomputes as much as possible.
### Does this PR introduce _any_ user-facing change?
Hopefully their jobs run faster or at the same speed. It also enables experimental shuffle service free dynamic scaling when graceful decommissioning is enabled (using the same code as the shuffle tracking dynamic scaling).
### How was this patch tested?
For now I've extended the ExecutorAllocationManagerSuite for both core & streaming.
Closes#29367 from holdenk/SPARK-31198-use-graceful-decommissioning-as-part-of-dynamic-scaling.
Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@apple.com>
### What changes were proposed in this pull request?
Set more dispatcher threads for the flaky test.
### Why are the changes needed?
When running test on Github Actions machine, the available processors in JVM is only 2, while on Jenkins it's 32. For this specific test, 2 available processors, which also decides the number of threads in Dispatcher, are not enough to consume the messages. In the worst situation, `MockExecutorLaunchFailWorker` would occupy these 2 threads for handling messages `LaunchDriver`, `LaunchExecutor` at the same time but leave no thread for the driver to handle the message `RegisteredApplication`. At the end, it results in a deadlock situation and causes the test failure.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
We can check whether the test is still flaky in Github Actions after this fix.
Closes#29408 from Ngone51/spark-32250.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
<!--
Thanks for sending a pull request! Here are some tips for you:
1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
4. Be sure to keep the PR description updated to reflect all changes.
5. Please write your PR title to summarize what this PR proposes.
6. If possible, provide a concise example to reproduce the issue for a faster review.
7. If you want to add a new configuration, please read the guideline first for naming configurations in
'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
-->
### What changes were proposed in this pull request?
Clear Ivy resolution files as part of finally block if not failures while artifacts resolution can leave the resolution files around.
Use tempIvyPath for SparkSubmitUtils.buildIvySettings in tests. This why the test
"SPARK-10878: test resolution files cleaned after resolving artifact" did not capture these issues.
### Why are the changes needed?
This is a bug
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Existing unit tests
Closes#29411 from venkata91/SPARK-32596.
Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
### What changes were proposed in this pull request?
This PR removes the total task result size check for shuffle map stage tasks, as these tasks return map status and metrics, which will not be cached on the driver and thus will not crash the driver.
### Why are the changes needed?
Checking total task result size for shuffle map stage tasks would lead to erroring normal jobs which create a big number of tasks even if the job eventually does not return a large dataset.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added UT.
Closes#29276 from maryannxue/spark-32470.
Authored-by: Maryann Xue <maryann.xue@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR aims to add `StorageLevel.DISK_ONLY_3` as a built-in `StorageLevel`.
### Why are the changes needed?
In a YARN cluster, HDFS uaually provides storages with replication factor 3. So, we can save the result to HDFS to get `StorageLevel.DISK_ONLY_3` technically. However, disaggregate clusters or clusters without storage services are rising. Previously, in that situation, the users were able to use similar `MEMORY_AND_DISK_2` or a user-created `StorageLevel`. This PR aims to support those use cases officially for better UX.
### Does this PR introduce _any_ user-facing change?
Yes. This provides a new built-in option.
### How was this patch tested?
Pass the GitHub Action or Jenkins with the revised test cases.
Closes#29331 from dongjoon-hyun/SPARK-32517.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
As reported by HyukjinKwon, BlockManagerDecommissionIntegrationSuite test is apparently still flaky (even after https://github.com/apache/spark/pull/29226): https://github.com/apache/spark/pull/29226#issuecomment-670286829.
The new flakyness is because the executors are not launching in the 6 seconds time out I had given them when run under github checks.
Bumped up the timeouts.
### Why are the changes needed?
To make this test not flaky so that it can give us high signal if decommissioning regresses.
### Does this PR introduce _any_ user-facing change?
No, unit test only check.
### How was this patch tested?
No new tests. Just github and jenkins.
Closes#29388 from agrawaldevesh/more_bm_harden.
Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR adds a try catch wrapping the History server scan logic to log and swallow the exception per entry.
### Why are the changes needed?
As discussed in #29350 , one entry failure shouldn't affect others.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manually tested.
Closes#29374 from yanxiaole/SPARK-32557.
Authored-by: Yan Xiaole <xiaole.yan@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR proposes to change the behavior of DataTable for stage-page and executors-page not to save the previous search text.
### Why are the changes needed?
DataTable is used in stage-page and executors-page for pagination and filter tasks/executors by search text.
In the current implementation, search text is saved so if we visit stage-page for a job, the previous search text is filled in the textbox and the task table is filtered.
I'm sometimes surprised by this behavior as the stage-page lists no tasks because tasks are filtered by the previous search text.
I think, it's not useful.
### Does this PR introduce _any_ user-facing change?
Yes. Search text is no longer saved.
### How was this patch tested?
New testcase with the following command.
```
$ build/sbt -Dguava.version=27.0-jre -Dtest.default.exclude.tags= -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite -- -z Search"
```
Closes#29265 from sarutak/fix-search-box.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR fixes some typos in <code>core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala</code> file.
### Why are the changes needed?
<code>spark.dynamicAllocation.sustainedSchedulerBacklogTimeout</code> (N) is used only after the <code>spark.dynamicAllocation.schedulerBacklogTimeout</code> (M) is exceeded.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
No test needed.
Closes#29351 from JoeyValentine/master.
Authored-by: JoeyValentine <rlaalsdn0506@naver.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR lets the logger log timestamp based on local time zone during test.
`SparkFunSuite` fixes the default time zone to America/Los_Angeles so the timestamp logged in unit-tests.log is also based on the fixed time zone.
### Why are the changes needed?
It's confusable for developers whose time zone is not America/Los_Angeles.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Run existing tests and confirmed uint-tests.log.
If your local time zone is America/Los_Angeles, you can test by setting the environment variable `TZ` like as follows.
```
$ TZ=Asia/Tokyo build/sbt "testOnly org.apache.spark.executor.ExecutorSuite"
$ tail core/target/unit-tests.log
```
Closes#29356 from sarutak/fix-unit-test-log-timezone.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Disallow `FileSystem.get(Configuration conf)` in Scala style check by default and suggest developers use `FileSystem.get(URI uri, Configuration conf)` or `Path.getFileSystem()` instead.
### Why are the changes needed?
The method `FileSystem.get(Configuration conf)` will return a default FileSystem instance if the conf `fs.file.impl` is not set. This can cause file not found exception on reading a target path of non-default file system, e.g. S3. It is hard to discover such a mistake via unit tests.
If we disallow it in Scala style check by default and suggest developers use `FileSystem.get(URI uri, Configuration conf)` or `Path.getFileSystem(Configuration conf)`, we can reduce potential regression and PR review effort.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manually run scala style check and test.
Closes#29357 from gengliangwang/newStyleRule.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
1. Make `CoarseGrainedSchedulerBackend.maxNumConcurrentTasks()` considers all kinds of resources when calculating the max concurrent tasks
2. Refactor `calculateAvailableSlots()` to make it be able to be used for both `CoarseGrainedSchedulerBackend` and `TaskSchedulerImpl`
### Why are the changes needed?
Currently, `CoarseGrainedSchedulerBackend.maxNumConcurrentTasks()` only considers the CPU for the max concurrent tasks. This can cause the application to hang when a barrier stage requires extra custom resources but the cluster doesn't have enough corresponding resources. Because, without the checking for other custom resources in `maxNumConcurrentTasks`, the barrier stage can be submitted to the `TaskSchedulerImpl`. But the `TaskSchedulerImpl` won't launch tasks for the barrier stage due to the insufficient task slots calculated by `TaskSchedulerImpl.calculateAvailableSlots` (which does check all kinds of resources).
The application hang issue can be reproduced by the added unit test.
### Does this PR introduce _any_ user-facing change?
Yes. In case of a barrier stage requires more custom resources than the cluster has, the application can get hang before this PR but can fail due to insufficient resources at the end after this PR.
### How was this patch tested?
Added a unit test.
Closes#29332 from Ngone51/fix-slots.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Exit the executor when it has been asked to decommission and there is nothing left for it to do.
This is a rebase of https://github.com/apache/spark/pull/28817
### Why are the changes needed?
If we want to use decommissioning in Spark's own scale down we should terminate the executor once finished.
Furthermore, in graceful shutdown it makes sense to release resources we no longer need if we've been asked to shutdown by the cluster manager instead of always holding the resources as long as possible.
### Does this PR introduce _any_ user-facing change?
The decommissioned executors will exit and the end of decommissioning. This is sort of a user facing change, however decommissioning hasn't been in any releases yet.
### How was this patch tested?
I changed the unit test to not send the executor exit message and still wait on the executor exited message.
Closes#29211 from holdenk/SPARK-31197-exit-execs-redone.
Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
# What changes were proposed in this pull request?
This PR adds a `FileNotFoundException` try catch block while adding a new entry to history server application listing to skip the non-existing path.
### Why are the changes needed?
If there are a large number (>100k) of applications log dir, listing the log dir will take a few seconds. After getting the path list some applications might have finished already, and the filename will change from `foo.inprogress` to `foo`.
It leads to a problem when adding an entry to the listing, querying file status like `fileSizeForLastIndex` will throw out a `FileNotFoundException` exception if the application was finished. And the exception will abort current loop, in a busy cluster, it will make history server couldn't list and load any application log.
```
20/08/03 15:17:23 ERROR FsHistoryProvider: Exception in checking for event log updates
java.io.FileNotFoundException: File does not exist: hdfs://xx/logs/spark/application_11111111111111.lz4.inprogress
at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1527)
at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1520)
at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1520)
at org.apache.spark.deploy.history.SingleFileEventLogFileReader.status$lzycompute(EventLogFileReaders.scala:170)
```
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
1. setup another script keeps changing the filename of applications under history log dir
2. launch the history server
3. check whether the `File does not exist` error log was gone.
Closes#29350 from yanxiaole/SPARK-32529.
Authored-by: Yan Xiaole <xiaole.yan@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This is a follow-up of #29278.
This PR changes the config name to switch allow/disallow `SparkContext` in executors as per the comment https://github.com/apache/spark/pull/29278#pullrequestreview-460256338.
### Why are the changes needed?
The config name `spark.executor.allowSparkContext` is more reasonable.
### Does this PR introduce _any_ user-facing change?
Yes, the config name is changed.
### How was this patch tested?
Updated tests.
Closes#29340 from ueshin/issues/SPARK-32160/change_config_name.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR modified the parser code to handle invalid usages of a SET/RESET command.
For example;
```
SET spark.sql.ansi.enabled true
```
The above SQL command does not change the configuration value and it just tries to display the value of the configuration
`spark.sql.ansi.enabled true`. This PR disallows using special characters including spaces in the configuration name and reports a user-friendly error instead. In the error message, it tells users a workaround to use quotes or a string literal if they still needs to specify a configuration with them.
Before this PR:
```
scala> sql("SET spark.sql.ansi.enabled true").show(1, -1)
+---------------------------+-----------+
|key |value |
+---------------------------+-----------+
|spark.sql.ansi.enabled true|<undefined>|
+---------------------------+-----------+
```
After this PR:
```
scala> sql("SET spark.sql.ansi.enabled true")
org.apache.spark.sql.catalyst.parser.ParseException:
Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, please use quotes, e.g., SET `ke y`=value.(line 1, pos 0)
== SQL ==
SET spark.sql.ansi.enabled true
^^^
```
### Why are the changes needed?
For better user-friendly errors.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added tests in `SparkSqlParserSuite`.
Closes#29146 from maropu/SPARK-32257.
Lead-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
When https is enabled for Spark UI, an HTTP request will be redirected as an encoded HTTPS URL: https://github.com/apache/spark/pull/10238/files#diff-f79a5ead735b3d0b34b6b94486918e1cR312
When we create the redirect url, we will call getRequestURI and getQueryString. Both two methods may return an encoded string. However, we pass them directly to the following URI constructor
```
URI(String scheme, String authority, String path, String query, String fragment)
```
As this URI constructor assumes both path and query parameters are decoded strings, it will encode them again. This makes the redirect URL encoded twice.
This problem is on stage page with HTTPS enabled. The URL of "/taskTable" contains query parameter `order%5B0%5D%5Bcolumn%5D`. After encoded it becomes `order%255B0%255D%255Bcolumn%255D` and it will be decoded as `order%5B0%5D%5Bcolumn%5D` instead of `order[0][dir]`. When the parameter `order[0][dir]` is missing, there will be an excetpion from:
https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala#L176
and the stage page fail to load.
To fix the problem, we can try decoding the query parameters before encoding it. This is to make sure we encode the URL
### Why are the changes needed?
Fix a UI issue when HTTPS is enabled
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
A new Unit test + manually test on a cluster
Closes#29271 from gengliangwang/urlEncode.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
### What changes were proposed in this pull request?
Just few log lines fixes which are logging the object name instead of the stage IDs
### Why are the changes needed?
This would make it easier later for debugging.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Just log messages. Existing tests should be enough
Closes#29279 from venkata91/SPARK-31418-follow-up.
Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This is a follow-up of #28986.
This PR adds a config to switch allow/disallow to create `SparkContext` in executors.
- `spark.driver.allowSparkContextInExecutors`
### Why are the changes needed?
Some users or libraries actually create `SparkContext` in executors.
We shouldn't break their workloads.
### Does this PR introduce _any_ user-facing change?
Yes, users will be able to create `SparkContext` in executors with the config enabled.
### How was this patch tested?
More tests are added.
Closes#29278 from ueshin/issues/SPARK-32160/add_configs.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR removes a test added in SPARK-32175(#29002).
### Why are the changes needed?
That test is flaky. It can be mitigated by increasing the timeout but it would rather be simpler to remove the test.
See also the [discussion](https://github.com/apache/spark/pull/29002#issuecomment-666746857).
### Does this PR introduce _any_ user-facing change?
No.
Closes#29314 from sarutak/remove-flaky-test.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
### What changes were proposed in this pull request?
This test tries to fix the flakyness of BlockManagerDecommissionIntegrationSuite.
### Description of the problem
Make the block manager decommissioning test be less flaky
An interesting failure happens when migrateDuring = true (and persist or shuffle is true):
- We schedule the job with tasks on executors 0, 1, 2.
- We wait 300 ms and decommission executor 0.
- If the task is not yet done on executor 0, it will now fail because
the block manager won't be able to save the block. This condition is
easy to trigger on a loaded machine where the github checks run.
- The task with retry on a different executor (1 or 2) and its shuffle
blocks will land there.
- No actual block migration happens here because the decommissioned
executor technically failed before it could even produce a block.
To remove the above race, this change replaces the fixed wait for 300 ms to wait for an actual task to succeed. When a task has succeeded, we know its blocks would have been written for sure and thus its executor would certainly be forced to migrate those blocks when it is decommissioned.
The change always decommissions an executor on which a real task finished successfully instead of picking the first executor. Because the system may choose to schedule nothing on the first executor and instead run the two tasks on one executor.
### Why are the changes needed?
I have had bad luck with BlockManagerDecommissionIntegrationSuite and it has failed several times on my PRs. So fixing it.
### Does this PR introduce _any_ user-facing change?
No, unit test only change.
### How was this patch tested?
Github checks. Ran this test 100 times, 10 at a time in parallel in a script.
Closes#29226 from agrawaldevesh/block-manager-decom-flaky.
Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
### What changes were proposed in this pull request?
This PR reduces the prospect of a job loss during decommissioning. It
fixes two holes in the current decommissioning framework:
- (a) Loss of decommissioned executors is not treated as a job failure:
We know that the decommissioned executor would be dying soon, so its death is
clearly not caused by the application.
- (b) Shuffle files on the decommissioned host are cleared when the
first fetch failure is detected from a decommissioned host: This is a
bit tricky in terms of when to clear the shuffle state ? Ideally you
want to clear it the millisecond before the shuffle service on the node
dies (or the executor dies when there is no external shuffle service) --
too soon and it could lead to some wastage and too late would lead to
fetch failures.
The approach here is to do this clearing when the very first fetch
failure is observed on the decommissioned block manager, without waiting for
other blocks to also signal a failure.
### Why are the changes needed?
Without them decommissioning a lot of executors at a time leads to job failures.
### Code overview
The task scheduler tracks the executors that were decommissioned along with their
`ExecutorDecommissionInfo`. This information is used by: (a) For handling a `ExecutorProcessLost` error, or (b) by the `DAGScheduler` when handling a fetch failure.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added a new unit test `DecommissionWorkerSuite` to test the new behavior by exercising the Master-Worker decommissioning. I chose to add a new test since the setup logic was quite different from the existing `WorkerDecommissionSuite`. I am open to changing the name of the newly added test suite :-)
### Questions for reviewers
- Should I add a feature flag to guard these two behaviors ? They seem safe to me that they should only get triggered by decommissioning, but you never know :-).
Closes#29014 from agrawaldevesh/decom_harden.
Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
### What changes were proposed in this pull request?
So far, we fixed many stuffs in `core` module. This PR fixes the remaining UT failures in Scala 2.13.
- `OneApplicationResource.environmentInfo` will return a deterministic result for `sparkProperties`, `hadoopProperties`, `systemProperties`, and `classpathEntries`.
- `SubmitRestProtocolSuite` has Scala 2.13 answer in addition to the existing Scala 2.12 answer, and uses the expected answer based on the Scala runtime version.
### Why are the changes needed?
To support Scala 2.13.
### Does this PR introduce _any_ user-facing change?
Yes, `environmentInfo` is changed, but this fixes the indeterministic behavior.
### How was this patch tested?
- Scala 2.12: Pass the Jenkins or GitHub Action
- Scala 2.13: Do the following.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13
```
**BEFORE**
```
Tests: succeeded 2612, failed 3, canceled 1, ignored 8, pending 0
*** 3 TESTS FAILED ***
```
**AFTER**
```
Tests: succeeded 2615, failed 0, canceled 1, ignored 8, pending 0
All tests passed.
```
Closes#29298 from dongjoon-hyun/SPARK-32489.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR aims to remove `java.ws.rs.NotFoundException` from two problematic `import` statements. All the other use cases are correct.
### Why are the changes needed?
In `StagesResource` and `OneApplicationResource`, there exist two `NotFoundException`s.
- javax.ws.rs.NotFoundException
- org.apache.spark.status.api.v1.NotFoundException
To use `org.apache.spark.status.api.v1.NotFoundException` correctly, we should not import `java.ws.rs.NotFoundException`. This causes UT failures in Scala 2.13 environment.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
- Scala 2.12: Pass the GitHub Action or Jenkins.
- Scala 2.13: Do the following manually.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.deploy.history.HistoryServerSuite
```
**BEFORE**
```
*** 4 TESTS FAILED ***
```
**AFTER**
```
*** 1 TEST FAILED ***
```
Closes#29293 from dongjoon-hyun/SPARK-32487.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR aims to make `ResourceAllocator.availableAddrs` deterministic.
### Why are the changes needed?
Currently, this function returns indeterministically due to the underlying `HashMap`. So, the test case itself is creating a list `[0, 1, 2]` initially, but ends up with comparing `[2, 1, 0]`.
Not only this happens in the 3.0.0, but also this causes UT failures on Scala 2.13 environment.
### Does this PR introduce _any_ user-facing change?
Yes, but this fixes the in-deterministic behavior.
### How was this patch tested?
- Scala 2.12: This should pass the UT with the modified test case.
- Scala 2.13: This can be tested like the following (at least `JsonProtocolSuite`)
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.deploy.JsonProtocolSuite
```
**BEFORE**
```
*** 2 TESTS FAILED ***
```
**AFTER**
```
All tests passed.
```
Closes#29281 from dongjoon-hyun/SPARK-32476.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR aims to make `JsonProtocol.accumulablesToJson` deterministic.
### Why are the changes needed?
Currently, `JsonProtocol.accumulablesToJson` is indeterministic. So, `JsonProtocolSuite` itself is also using mixed test cases in terms of `"Accumulables": [ ... ]`.
Not only this is indeterministic, but also this causes a UT failure in `JsonProtocolSuite` in Scala 2.13.
### Does this PR introduce _any_ user-facing change?
Yes. However, this is a fix on indeterministic behavior.
### How was this patch tested?
- Scala 2.12: Pass the GitHub Action or Jenkins.
- Scala 2.13: Do the following.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.util.JsonProtocolSuite
```
**BEFORE**
```
*** 1 TEST FAILED ***
```
**AFTER**
```
All tests passed.
```
Closes#29282 from dongjoon-hyun/SPARK-32477.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR changes the order between initialization for ExecutorPlugin and starting heartbeat thread in Executor.
### Why are the changes needed?
In the current master, heartbeat thread in a executor starts after plugin initialization so if the initialization takes long time, heartbeat is not sent to driver and the executor will be removed from cluster.
### Does this PR introduce _any_ user-facing change?
Yes. Plugins for executors will be allowed to take long time for initialization.
### How was this patch tested?
New testcase.
Closes#29002 from sarutak/fix-heartbeat-issue.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
### What changes were proposed in this pull request?
`spark.kryo.registrator` in 3.0 has a regression problem. From [SPARK-12080](https://issues.apache.org/jira/browse/SPARK-12080), it supports multiple user registrators by
```scala
private val userRegistrators = conf.get("spark.kryo.registrator", "")
.split(',').map(_.trim)
.filter(!_.isEmpty)
```
But it donsn't work in 3.0. Fix it by `toSequence` in `Kryo.scala`
### Why are the changes needed?
In previous Spark version (2.x), it supported multiple user registrators by
```scala
private val userRegistrators = conf.get("spark.kryo.registrator", "")
.split(',').map(_.trim)
.filter(!_.isEmpty)
```
But it doesn't work in 3.0. It's should be a regression.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Existed unit tests.
Closes#29123 from LantaoJin/SPARK-32283.
Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR aims to use `command -v` in non-Window operating systems instead of executing the given command.
### Why are the changes needed?
1. `command` is POSIX-compatible
- **POSIX.1-2017**: https://pubs.opengroup.org/onlinepubs/9699919799/utilities/command.html
2. `command` is faster and safer than the direct execution
- `command` doesn't invoke another process.
```scala
scala> sys.process.Process("ls").run().exitValue()
LICENSE
NOTICE
bin
doc
lib
man
res1: Int = 0
```
3. The existing way behaves inconsistently.
- `rm` cannot be checked.
**AS-IS**
```scala
scala> sys.process.Process("rm").run().exitValue()
usage: rm [-f | -i] [-dPRrvW] file ...
unlink file
res0: Int = 64
```
**TO-BE**
```
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.
scala> sys.process.Process(Seq("sh", "-c", s"command -v ls")).run().exitValue()
/bin/ls
val res1: Int = 0
```
4. The existing logic is already broken in Scala 2.13 environment because it hangs like the following.
```scala
$ bin/scala
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.
scala> sys.process.Process("cat").run().exitValue() // hang here.
```
### Does this PR introduce _any_ user-facing change?
No. Although this is inside `main` source directory, this is used for testing purpose.
```
$ git grep testCommandAvailable | grep -v 'def testCommandAvailable'
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("wc"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable(envCommand))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(!TestUtils.testCommandAvailable("some_nonexistent_command"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala: assume(TestUtils.testCommandAvailable(envCommand))
sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala: private lazy val isPythonAvailable: Boolean = TestUtils.testCommandAvailable(pythonExec)
sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala: if (TestUtils.testCommandAvailable(pythonExec)) {
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala: skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("python"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala: assume(TestUtils.testCommandAvailable("echo | sed"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala: assume(TestUtils.testCommandAvailable("/bin/bash"))
```
### How was this patch tested?
- **Scala 2.12**: Pass the Jenkins with the existing tests and one modified test.
- **Scala 2.13**: Do the following manually. It should pass instead of `hang`.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.rdd.PipedRDDSuite
...
Tests: succeeded 12, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```
Closes#29241 from dongjoon-hyun/SPARK-32443.
Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
Support set off heap memory in `ExecutorResourceRequests`
### Why are the changes needed?
Support stage level scheduling
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added UT in `ResourceProfileSuite` and `DAGSchedulerSuite`
Closes#28972 from warrenzhu25/30794.
Authored-by: Warren Zhu <zhonzh@microsoft.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
### What changes were proposed in this pull request?
This PR aims to increase the memory parameter in `BlockManagerSuite`'s worker decommission test cases.
### Why are the changes needed?
Scala 2.13 generates different Java objects and this affects Spark's `SizeEstimator/SizeTracker/SizeTrackingVector`. This causes UT failures like the following. If we decrease the values, those test cases fails in Scala 2.12, too.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.storage.BlockManagerSuite
...
- test decommission block manager should not be part of peers *** FAILED ***
0 did not equal 2 (BlockManagerSuite.scala:1869)
- test decommissionRddCacheBlocks should offload all cached blocks *** FAILED ***
0 did not equal 2 (BlockManagerSuite.scala:1884)
...
Tests: succeeded 81, failed 2, canceled 0, ignored 0, pending 0
*** 2 TESTS FAILED ***
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.storage.BlockManagerSuite
...
Tests: succeeded 83, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```
Closes#29238 from dongjoon-hyun/SPARK-32440.
Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>