Commit graph

7847 commits

Author SHA1 Message Date
“attilapiros” 6bd7a6200f [SPARK-33711][K8S] Avoid race condition between POD lifecycle manager and scheduler backend
### 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>
2021-01-11 14:25:12 -08:00
yikf 1495ad8c46 [SPARK-33991][CORE][WEBUI] Repair enumeration conversion error for AllJobsPage
### 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>
2021-01-11 08:48:02 -06:00
angerszhu 5ef6907792 [SPARK-33084][CORE][SQL] Rename Unit test file and use fake ivy link
### 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>
2021-01-11 23:18:18 +09:00
yi.wu 4afca0f706 [SPARK-31952][SQL] Fix incorrect memory spill metric when doing Aggregate
### 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>
2021-01-11 07:15:28 +00:00
HyukjinKwon 830249284d [SPARK-34059][SQL][CORE] Use for/foreach rather than map to make sure execute it eagerly
### 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>
2021-01-10 15:22:24 -08:00
Chandni Singh d00f0695b7 [SPARK-32917][SHUFFLE][CORE] Adds support for executors to push shuffle blocks after successful map task completion
### 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>
2021-01-08 12:21:56 -06:00
Kousuke Saruta cc20154562 [SPARK-34005][CORE] Update peak memory metrics for each Executor on task end
### 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>
2021-01-07 21:24:15 -08:00
Baohe Zhang 29510821a0 [SPARK-33029][CORE][WEBUI] Fix the UI executor page incorrectly marking the driver as excluded
### 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>
2021-01-05 19:16:40 -08:00
LantaoJin a7d3fcd354 [SPARK-34000][CORE] Fix stageAttemptToNumSpeculativeTasks java.util.NoSuchElementException
### 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>
2021-01-04 21:37:26 -08:00
angerszhu 559f411da8 [SPARK-33908][CORE][FOLLOWUP] Correct Scaladoc of resolveDependencyPaths/resolveMavenDependencies
### 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>
2021-01-04 15:44:42 -08:00
HyukjinKwon 6b86aa0b52 [SPARK-33984][PYTHON] Upgrade to Py4J 0.10.9.1
### 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>
2021-01-04 10:23:38 -08:00
Baohe Zhang 45df6db906 [SPARK-33906][WEBUI] Fix the bug of UI Executor page stuck due to undefined peakMemoryMetrics
### 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>
2020-12-31 13:34:55 -08:00
yi.wu 3fe5614a7c [SPARK-31946][CORE] Make worker/executor decommission signal configurable
### 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>
2020-12-31 13:13:02 -08:00
yangjie01 85de644733 [SPARK-33804][CORE] Fix compilation warnings about 'view bounds are deprecated'
### 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>
2020-12-30 13:57:44 -06:00
yi.wu 1ef7ddd38a [SPARK-33928][SPARK-23365][TEST][CORE] Fix flaky o.a.s.ExecutorAllocationManagerSuite - " Don't update target num executors when killing idle executors"
### 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>
2020-12-29 07:35:45 +00:00
HyukjinKwon b33fa53385 [SPARK-33925][CORE] Remove unused SecurityManager in Utils.fetchFile
### 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>
2020-12-28 16:58:42 -08:00
Dongjoon Hyun 6497ccbbda [SPARK-33916][CORE] Fix fallback storage offset and improve compression codec test coverage
### 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>
2020-12-28 16:33:01 -08:00
angerszhu 0a3f3d609d [SPARK-33908][CORE] Refactor SparkSubmitUtils.resolveMavenCoordinates() 's return parameter
### 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>
2020-12-28 16:00:24 +09:00
yangjie01 1be9e7e40b [SPAKR-33801][CORE][SQL] Fix compilation warnings about 'Unicode escapes in triple quoted strings are deprecated'
### 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>
2020-12-28 15:29:09 +09:00
angerszhu 10b6466e91 [SPARK-33084][CORE][SQL] Add jar support ivy path
### 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>
2020-12-25 09:07:48 +09:00
sychen 700f5ab65c [SPARK-33900][WEBUI] Show shuffle read size / records correctly when only remotebytesread is available
### What changes were proposed in this pull request?
Shuffle Read Size / Records can also be displayed in remoteBytesRead>0 localBytesRead=0.

current:
![image](https://user-images.githubusercontent.com/3898450/103079421-c4ca2280-460e-11eb-9e2f-49d35b5d324d.png)
fix:
![image](https://user-images.githubusercontent.com/3898450/103079439-cc89c700-460e-11eb-9a41-6b2882980d11.png)

### Why are the changes needed?
At present, the page only displays the data of Shuffle Read Size / Records when localBytesRead>0.
When there is only remote reading, metrics cannot be seen on the stage page.

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

### How was this patch tested?
manual test

Closes #30916 from cxzl25/SPARK-33900.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-12-25 00:54:26 +09:00
Dongjoon Hyun d467d81726 [SPARK-33893][CORE] Exclude fallback block manager from executorList
### 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>
2020-12-23 15:31:56 -08:00
Takuya UESHIN 5c9b421c37 [SPARK-33277][PYSPARK][SQL] Use ContextAwareIterator to stop consuming after the task ends
### 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>
2020-12-23 14:48:01 -08:00
Dongjoon Hyun 90d6f86001 [SPARK-33870][CORE] Enable spark.storage.replication.proactive by default
### 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>
2020-12-22 21:59:53 -08:00
Dongjoon Hyun 16ae3a5c12 [MINOR][CORE] Remove unused variable CompressionCodec.DEFAULT_COMPRESSION_CODEC
### 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>
2020-12-21 19:48:58 -08:00
Dongjoon Hyun f62e957b31 [SPARK-33873][CORE][TESTS] Test all compression codecs with encrypted spilling
### 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>
2020-12-21 16:35:04 -08:00
Kousuke Saruta 8e2633962f [SPARK-26341][WEBUI][FOLLOWUP] Update stage memory metrics on stage end
### 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>
2020-12-20 17:38:47 -08:00
Xianjin YE 13391683e7 [SPARK-33756][SQL] Make BytesToBytesMap's MapIterator idempotent
### 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>
2020-12-20 08:51:17 -06:00
Vlad Glinsky 554600c2af
[SPARK-33841][CORE] Fix issue with jobs disappearing intermittently from the SHS under high load
### What changes were proposed in this pull request?

Mark SHS event log entries that were `processing` at the beginning of the `checkForLogs` run as not stale and check for this mark before deleting an event log. This fixes the issue when a particular job was displayed in the SHS and disappeared after some time, but then, in several minutes showed up again.

### Why are the changes needed?

The issue is caused by [SPARK-29043](https://issues.apache.org/jira/browse/SPARK-29043), which is designated to improve the concurrent performance of the History Server. The [change](https://github.com/apache/spark/pull/25797/files#) breaks the ["app deletion" logic](https://github.com/apache/spark/pull/25797/files#diff-128a6af0d78f4a6180774faedb335d6168dfc4defff58f5aa3021fc1bd767bc0R563) because of missing proper synchronization for `processing` event log entries. Since SHS now [filters out](https://github.com/apache/spark/pull/25797/files#diff-128a6af0d78f4a6180774faedb335d6168dfc4defff58f5aa3021fc1bd767bc0R462) all `processing` event log entries, such entries do not have a chance to be [updated with the new `lastProcessed`](https://github.com/apache/spark/pull/25797/files#diff-128a6af0d78f4a6180774faedb335d6168dfc4defff58f5aa3021fc1bd767bc0R472) time and thus any entity that completes processing right after [filtering](https://github.com/apache/spark/pull/25797/files#diff-128a6af0d78f4a6180774faedb335d6168dfc4defff58f5aa3021fc1bd767bc0R462) and before [the check for stale entities](https://github.com/apache/spark/pull/25797/files#diff-128a6af0d78f4a6180774faedb335d6168dfc4defff58f5aa3021fc1bd767bc0R560) will be identified as stale and will be deleted from the UI until the next `checkForLogs` run. This is because [updated `lastProcessed` time is used as criteria](https://github.com/apache/spark/pull/25797/files#diff-128a6af0d78f4a6180774faedb335d6168dfc4defff58f5aa3021fc1bd767bc0R557), and event log entries that missed to be updated with a new time, will match that criteria.

The issue can be reproduced by generating a big number of event logs and uploading them to the SHS event log directory on S3. Essentially, around 236(26.7 MB) copies of an event log directory were created using [shs-monitor](https://github.com/vladhlinsky/shs-monitor/tree/spark-master) script. Strange behavior of SHS counting the total number of applications was noticed - at first, the number was increasing as expected, but with the next page refresh, the total number of applications decreased. No errors were logged by SHS.

58 entities are displayed at `17:35:35`:
![1-58-entries-at-17-35](https://user-images.githubusercontent.com/61428392/102648949-1129e400-4171-11eb-9463-ed1454a8f6b2.png)
25 entities are displayed at `17:36:40`:
![2-25-entries-at-17-36](https://user-images.githubusercontent.com/61428392/102648974-1c7d0f80-4171-11eb-95d8-78c2bb37a168.png)

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

Yes, SHS users won't face the behavior when the number of displayed applications decreases periodically.

### How was this patch tested?

Tested using [shs-monitor](https://github.com/vladhlinsky/shs-monitor/tree/spark-master) script:
* Build SHS with the proposed change
* Download Hadoop AWS and AWS Java SDK
* Prepare S3 bucket and user for programmatic access, grant required roles to the user. Get access key and secret key
* Configure SHS to read event logs from S3
* Start [monitor](https://github.com/vladhlinsky/shs-monitor/blob/spark-master/monitor.sh) script to query SHS API
* Run 5 [producers](https://github.com/vladhlinsky/shs-monitor/blob/spark-master/producer.sh) for ~5 mins, create 125(14.2 MB) event log directory copies
* Wait for SHS to load all the applications
* Verify that the number of loaded applications increases continuously over time

For more details, please refer to the [shs-monitor](https://github.com/vladhlinsky/shs-monitor/tree/spark-master) repository.
> This version of the reproduction uses event log directories instead of single files, since recent optimization
> [SPARK-33790](https://issues.apache.org/jira/browse/SPARK-33790) makes it hard to reproduce the issue with single event log files.

Closes #30845 from vladhlinsky/SPARK-33841.

Authored-by: Vlad Glinsky <vladhlinsky@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-18 13:26:19 -08:00
angerszhu 25c6cc25f7 [SPARK-26341][WEBUI] Expose executor memory metrics at the stage level, in the Stages tab
### What changes were proposed in this pull request?
Expose executor memory metrics at the stage level, in the Stages tab,
Current like below, and I am not sure which column we will truly need.
![image](https://user-images.githubusercontent.com/46485123/101170248-2256f900-3679-11eb-8c34-794fcf8e94a8.png)

![image](https://user-images.githubusercontent.com/46485123/101170359-4dd9e380-3679-11eb-984b-b0430f236160.png)

![image](https://user-images.githubusercontent.com/46485123/101314915-86a1d480-3894-11eb-9b6f-8050d326e11f.png)

### Why are the changes needed?
User can know executor jvm usage more directly in SparkUI

### Does this PR introduce any user-facing change?
User can know executor jvm usage more directly in SparkUI

### How was this patch tested?
Manual Tested

Closes #30573 from AngersZhuuuu/SPARK-26341.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-12-18 14:24:53 +09:00
yi.wu 15616f499a
[SPARK-33173][CORE][TESTS][FOLLOWUP] Use local[2] and AtomicInteger
### 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>
2020-12-17 09:28:17 -08:00
suqilong 8c81cf7d71 [SPARK-22769] Do not log rpc post message error when sparkEnv is already stopped
### 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>
2020-12-17 08:56:45 -06:00
yi.wu 34e4d87023 [SPARK-33774][UI][CORE] Back to Master" returns 500 error in Standalone cluster
### 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>
2020-12-17 08:52:09 -06:00
Dongjoon Hyun ed09673fb9 [SPARK-33819][CORE] SingleFileEventLogFileReader/RollingEventLogFilesFileReader should be package private
### 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>
2020-12-17 15:51:04 +09:00
sychen 0c12900120 [SPARK-33790][CORE] Reduce the rpc call of getFileStatus in SingleFileEventLogFileReader
### 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>
2020-12-17 11:36:31 +09:00
Sander Goos ddff94fd32 [SPARK-33793][TESTS] Introduce withExecutor to ensure proper cleanup in tests
### 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>
2020-12-16 11:26:54 +09:00
HyukjinKwon a99a47ca1d [SPARK-33748][K8S] Respect environment variables and configurations for Python executables
### 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>
2020-12-15 08:56:45 +09:00
Kent Yao 4d47ac4b4b [SPARK-33705][SQL][TEST] Fix HiveThriftHttpServerSuite flakiness
### 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>
2020-12-14 05:14:38 +00:00
linzebing 0277fddaef [MINOR][UI] Correct JobPage's skipped/pending tableHeaderId
### 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>
2020-12-13 22:00:05 +09:00
HyukjinKwon e2cdfcebd9 [SPARK-32447][CORE][PYTHON][FOLLOW-UP] Fix other occurrences of 'python' to 'python3'
### 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>
2020-12-13 10:41:47 +09:00
Dongjoon Hyun de9818f043
[SPARK-33662][BUILD] Setting version to 3.2.0-SNAPSHOT
### 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>
2020-12-04 14:10:42 -08:00
HyukjinKwon 990bee9c58 [SPARK-33615][K8S] Make 'spark.archives' working in Kubernates
### 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>
2020-12-04 19:37:03 +09:00
Gabor Somogyi bd711863fd [SPARK-33629][PYTHON] Make spark.buffer.size configuration visible on driver side
### 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>
2020-12-04 01:37:44 +09:00
yangjie01 92bfbcb2e3
[SPARK-33631][DOCS][TEST] Clean up spark.core.connection.ack.wait.timeout from configuration.md
### 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>
2020-12-02 12:58:41 -08:00
neko 28dad1ba77 [SPARK-33504][CORE] The application log in the Spark history server contains sensitive attributes should be redacted
### 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>
2020-12-02 09:24:19 -06:00
yangjie01 084d38b64e [SPARK-33557][CORE][MESOS][TEST] Ensure the relationship between STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT and NETWORK_TIMEOUT
### 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>
2020-12-02 18:41:49 +09:00
Dongjoon Hyun 290aa02179 [SPARK-33618][CORE] Use hadoop-client instead of hadoop-client-api to make hadoop-aws work
### 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>
2020-12-02 18:23:48 +09:00
Gengliang Wang 5d0045eedf [SPARK-33611][UI] Avoid encoding twice on the query parameter of rewritten proxy URL
### 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>
2020-12-02 01:36:41 +08:00
HyukjinKwon 1a042cc414 [SPARK-33530][CORE] Support --archives and spark.archives option natively
### 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>
2020-12-01 13:43:02 +09:00
Jungtaek Lim (HeartSaVioR) f5d2165c95 [SPARK-33440][CORE] Use current timestamp with warning log in HadoopFSDelegationTokenProvider when the issue date for token is not set up properly
### 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>
2020-12-01 06:44:15 +09:00