Commit graph

606 commits

Author SHA1 Message Date
Holden Karau 50641d2e3d [SPARK-34104][SPARK-34105][CORE][K8S] Maximum decommissioning time & allow decommissioning for excludes
### 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>
2021-02-09 14:21:24 -08:00
“attilapiros” b2dc38b654 [SPARK-34334][K8S] Correctly identify timed out pending pod requests as excess request
### What changes were proposed in this pull request?

Fixing identification of timed-out pending pod requests as excess requests to delete when the excess is higher than the newly created timed out requests and there is some non-timed out newly created requests too.

### Why are the changes needed?

After https://github.com/apache/spark/pull/29981 only timed out newly created requests and timed out pending requests are taken as excess request.

But there is small bug when the excess is higher than the newly created timed out requests and there is some non-timed out newly created requests as well. Because all the newly created requests are counted as excess request when items are chosen from the timed out pod pending requests.

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

No.

### How was this patch tested?

 There is new unit test added: `SPARK-34334: correctly identify timed out pending pod requests as excess`.

Closes #31445 from attilapiros/SPARK-34334.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2021-02-09 10:06:55 -08:00
Dongjoon Hyun ea339c38b4 [SPARK-34407][K8S] KubernetesClusterSchedulerBackend.stop should clean up K8s resources
### What changes were proposed in this pull request?

This PR aims to fix `KubernetesClusterSchedulerBackend.stop` to wrap `super.stop` with `Utils.tryLogNonFatalError`.

### Why are the changes needed?

[CoarseGrainedSchedulerBackend.stop](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala#L559) may throw `SparkException` and this causes K8s resource (pod and configmap) leakage.

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

No. This is a bug fix.

### How was this patch tested?

Pass the CI with the newly added test case.

Closes #31533 from dongjoon-hyun/SPARK-34407.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-08 21:47:23 -08:00
yangjie01 b344e91368 [SPARK-34375][CORE][K8S][TEST] Replaces 'Mockito.initMocks' with 'Mockito.openMocks'
### 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>
2021-02-08 15:13:00 +09:00
Dongjoon Hyun f66e38c963 [SPARK-34316][K8S] Support spark.kubernetes.executor.disableConfigMap
### What changes were proposed in this pull request?

This PR aims to add a new configuration `spark.kubernetes.executor.disableConfigMap`.

### Why are the changes needed?

This can be use to disable config map creating for executor pods due to https://github.com/apache/spark/pull/27735 .

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

No. By default, this doesn't change AS-IS behavior.
This is a new feature to add an ability to disable SPARK-30985.

### How was this patch tested?

Pass the newly added UT.

Closes #31428 from dongjoon-hyun/SPARK-34316.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-01 22:26:07 -08:00
“attilapiros” d3f049cbc2 [SPARK-34154][YARN][FOLLOWUP] Fix flaky LocalityPlacementStrategySuite test
### What changes were proposed in this pull request?

Fixing the flaky `handle large number of containers and tasks (SPARK-18750)` by avoiding to use `DNSToSwitchMapping` as in some situation DNS lookup could be extremely slow.

### Why are the changes needed?

After https://github.com/apache/spark/pull/31363 was merged the flaky `handle large number of containers and tasks (SPARK-18750)` test failed again in some other PRs but now we have the exact place where the test is stuck.

It is in the DNS lookup:

```
[info] - handle large number of containers and tasks (SPARK-18750) *** FAILED *** (30 seconds, 4 milliseconds)
[info]   Failed with an exception or a timeout at thread join:
[info]
[info]   java.lang.RuntimeException: Timeout at waiting for thread to stop (its stack trace is added to the exception)
[info]   	at java.net.Inet6AddressImpl.lookupAllHostAddr(Native Method)
[info]   	at java.net.InetAddress$2.lookupAllHostAddr(InetAddress.java:929)
[info]   	at java.net.InetAddress.getAddressesFromNameService(InetAddress.java:1324)
[info]   	at java.net.InetAddress.getAllByName0(InetAddress.java:1277)
[info]   	at java.net.InetAddress.getAllByName(InetAddress.java:1193)
[info]   	at java.net.InetAddress.getAllByName(InetAddress.java:1127)
[info]   	at java.net.InetAddress.getByName(InetAddress.java:1077)
[info]   	at org.apache.hadoop.net.NetUtils.normalizeHostName(NetUtils.java:568)
[info]   	at org.apache.hadoop.net.NetUtils.normalizeHostNames(NetUtils.java:585)
[info]   	at org.apache.hadoop.net.CachedDNSToSwitchMapping.resolve(CachedDNSToSwitchMapping.java:109)
[info]   	at org.apache.spark.deploy.yarn.SparkRackResolver.coreResolve(SparkRackResolver.scala:75)
[info]   	at org.apache.spark.deploy.yarn.SparkRackResolver.resolve(SparkRackResolver.scala:66)
[info]   	at org.apache.spark.deploy.yarn.LocalityPreferredContainerPlacementStrategy.$anonfun$localityOfRequestedContainers$3(LocalityPreferredContainerPlacementStrategy.scala:142)
[info]   	at org.apache.spark.deploy.yarn.LocalityPreferredContainerPlacementStrategy$$Lambda$658/1080992036.apply$mcVI$sp(Unknown Source)
[info]   	at scala.collection.immutable.Range.foreach$mVc$sp(Range.scala:158)
[info]   	at org.apache.spark.deploy.yarn.LocalityPreferredContainerPlacementStrategy.localityOfRequestedContainers(LocalityPreferredContainerPlacementStrategy.scala:138)
[info]   	at org.apache.spark.deploy.yarn.LocalityPlacementStrategySuite.org$apache$spark$deploy$yarn$LocalityPlacementStrategySuite$$runTest(LocalityPlacementStrategySuite.scala:94)
[info]   	at org.apache.spark.deploy.yarn.LocalityPlacementStrategySuite$$anon$1.run(LocalityPlacementStrategySuite.scala:40)
[info]   	at java.lang.Thread.run(Thread.java:748) (LocalityPlacementStrategySuite.scala:61)
...
```

This could be because of the DNS servers used by those build machines are not configured to handle IPv6 queries and the client has to wait for the IPv6 query to timeout before falling back to IPv4.

This even make the tests more consistent. As when a single host was given to lookup via `resolve(hostName: String)` it gave a different answer from calling `resolve(hostNames: Seq[String])` with a `Seq` containing that single host.

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

No.

### How was this patch tested?

Unit tests.

Closes #31397 from attilapiros/SPARK-34154-2nd.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-29 23:54:40 +09:00
Dongjoon Hyun 78244bafe8 [SPARK-34281][K8S] Promote spark.kubernetes.executor.podNamePrefix to the public conf
### What changes were proposed in this pull request?

This PR aims to remove `internal()` from `spark.kubernetes.executor.podNamePrefix` in order to make it the configuration public.

### Why are the changes needed?

In line with K8s GA, this will allow some users control the full executor pod names officially.
This is useful when we want a custom executor pod name pattern independently from the app name.

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

No, this has been there since Apache Spark 2.3.0.

### How was this patch tested?

N/A.

Closes #31386 from dongjoon-hyun/SPARK-34281.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-28 13:01:18 -08:00
“attilapiros” 0dedf24cd0 [SPARK-34154][YARN] Extend LocalityPlacementStrategySuite's test with a timeout
### What changes were proposed in this pull request?

This PR extends the `handle large number of containers and tasks (SPARK-18750)` test with a time limit and in case of timeout it saves the stack trace of the running thread to provide extra information about the reason why it got stuck.

### Why are the changes needed?

This is a flaky test which sometime runs for hours without stopping.

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

No.

### How was this patch tested?

I checked it with a temporary code change: by adding a `Thread.sleep` to `LocalityPreferredContainerPlacementStrategy#expectedHostToContainerCount`.

The stack trace showed the correct method:

```
[info] LocalityPlacementStrategySuite:
[info] - handle large number of containers and tasks (SPARK-18750) *** FAILED *** (30 seconds, 26 milliseconds)
[info]   Failed with an exception or a timeout at thread join:
[info]
[info]   java.lang.RuntimeException: Timeout at waiting for thread to stop (its stack trace is added to the exception)
[info]   	at java.lang.Thread.sleep(Native Method)
[info]   	at org.apache.spark.deploy.yarn.LocalityPreferredContainerPlacementStrategy.$anonfun$expectedHostToContainerCount$1(LocalityPreferredContainerPlacementStrategy.scala:198)
[info]   	at org.apache.spark.deploy.yarn.LocalityPreferredContainerPlacementStrategy$$Lambda$281/381161906.apply(Unknown Source)
[info]   	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
[info]   	at scala.collection.TraversableLike$$Lambda$16/322836221.apply(Unknown Source)
[info]   	at scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:234)
[info]   	at scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:468)
[info]   	at scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:468)
[info]   	at scala.collection.TraversableLike.map(TraversableLike.scala:238)
[info]   	at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
[info]   	at scala.collection.AbstractTraversable.map(Traversable.scala:108)
[info]   	at org.apache.spark.deploy.yarn.LocalityPreferredContainerPlacementStrategy.expectedHostToContainerCount(LocalityPreferredContainerPlacementStrategy.scala:188)
[info]   	at org.apache.spark.deploy.yarn.LocalityPreferredContainerPlacementStrategy.localityOfRequestedContainers(LocalityPreferredContainerPlacementStrategy.scala:112)
[info]   	at org.apache.spark.deploy.yarn.LocalityPlacementStrategySuite.org$apache$spark$deploy$yarn$LocalityPlacementStrategySuite$$runTest(LocalityPlacementStrategySuite.scala:94)
[info]   	at org.apache.spark.deploy.yarn.LocalityPlacementStrategySuite$$anon$1.run(LocalityPlacementStrategySuite.scala:40)
[info]   	at java.lang.Thread.run(Thread.java:748) (LocalityPlacementStrategySuite.scala:61)
...
```

Closes #31363 from attilapiros/SPARK-34154.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 08:04:25 +09:00
yangjie01 8999e8805d [SPARK-34224][CORE][SQL][SS][DSTREAM][YARN][TEST][EXAMPLES] Ensure all resource opened by Source.fromXXX are closed
### 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>
2021-01-26 19:06:37 +09:00
Dongjoon Hyun 134a7d7eb9 [SPARK-34206][K8S] Make Guava Cache as ExecutorPodsLifecycleManager private field
### What changes were proposed in this pull request?

`KubernetesClusterManager` and `ExecutorPodsLifecycleManager` are private Spark classes.
This PR aims to move `Guava Cache` from a constructor parameter to private field of `ExecutorPodsLifecycleManager`.

### Why are the changes needed?

1. Although `KubernetesClusterManager` creates `Guava Cache`, only `ExecutorPodsLifecycleManager` uses it.
2. Although `ExecutorPodsLifecycleManager` is a Spark private class, when some users implement a new cluster manager with `ExternalClusterManager` for K8s, they can reuse `ExecutorPodsLifecycleManager`. In this case, `Guava Cache` is not good as an interface because it's a shaded class.

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

No. This is an Spark private.

### How was this patch tested?

Pass the existing UTs.

Closes #31297 from dongjoon-hyun/SPARK-34206.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-22 19:36:07 -08:00
Chao Sun b6f46ca297 [SPARK-33212][BUILD] Upgrade to Hadoop 3.2.2 and move to shaded clients for Hadoop 3.x profile
### 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>
2021-01-15 14:06:50 -08:00
yangjie01 8b1ba233f1 [SPARK-34068][CORE][SQL][MLLIB][GRAPHX] Remove redundant collection conversion
### 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>
2021-01-13 18:07:02 -06:00
“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
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
Holden Karau 8e11ce5378 [SPARK-34018][K8S] NPE in ExecutorPodsSnapshot
### What changes were proposed in this pull request?

Label both the statuses and ensure the ExecutorPodSnapshot starts with the default config to match.

### Why are the changes needed?

The current test depends on the order rather than testing the desired property.

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

No

### How was this patch tested?

Labeled the containers statuses, observed failures, added the default label as the initialization point, tests passed again.

Built Spark, ran on K8s cluster verified no NPE in driver log.

Closes #31071 from holdenk/SPARK-34018-finishedExecutorWithRunningSidecar-doesnt-correctly-constructt-the-test-case.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-07 16:47:37 -08:00
Prashant Sharma f64dfa8727 [SPARK-32221][K8S] Avoid possible errors due to incorrect file size or type supplied in spark conf
### What changes were proposed in this pull request?

Skip files if they are binary or very large to fit the configMap's max size.

### Why are the changes needed?

Config map cannot hold binary files and there is also a limit on how much data a configMap can hold.
This limit can be configured by the k8s cluster admin. This PR, skips such files (with a warning) instead of failing with weird runtime errors.
If such files are not skipped, then it would result in mount errors or encoding errors (if binary files are submitted).

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

yes, in simple words avoids possible errors due to negligence (for example, placing a large file or a binary file in SPARK_CONF_DIR) and thus improves user experience.

### How was this patch tested?

Added relevant tests and improved existing tests.

Closes #30472 from ScrapCodes/SPARK-32221/avoid-conf-propagate-errors.

Lead-authored-by: Prashant Sharma <prashsh1@in.ibm.com>
Co-authored-by: Prashant Sharma <prashant@apache.org>
Signed-off-by: Prashant Sharma <prashsh1@in.ibm.com>
2021-01-06 14:55:40 +05:30
Holden Karau 171db85aa2 [SPARK-33874][K8S][FOLLOWUP] Handle long lived sidecars - clean up logging
### What changes were proposed in this pull request?

Switch log level from warn to debug when the spark container is not present in the pod's container statuses.

### Why are the changes needed?

There are many non-critical situations where the Spark container may not be present, and the warning log level is too high.

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

Log message change.

### How was this patch tested?

N/A

Closes #31047 from holdenk/SPARK-33874-follow-up.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-05 13:48:52 -08:00
Holden Karau 448494ebcf [SPARK-33874][K8S] Handle long lived sidecars
### What changes were proposed in this pull request?

For liveness check when checkAllContainers is not set, we check the liveness status of the Spark container if we can find it.

### Why are the changes needed?

Some environments may deploy long lived logs collecting side cars which outlive the Spark application. Just because they remain alive does not mean the Spark executor should keep running.

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

No

### How was this patch tested?

Extended the existing pod status tests.

Closes #30892 from holdenk/SPARK-33874-handle-long-lived-sidecars.

Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-30 14:06:34 +09:00
David McWhorter 87c58367cd [SPARK-22256][MESOS] Introduce spark.mesos.driver.memoryOverhead
### What changes were proposed in this pull request?
This is a simple change to support allocating a specified amount of overhead memory for the driver's mesos container.  This is already supported for executors.

### Why are the changes needed?
This is needed to keep the driver process from exceeding memory limits and being killed off when running on mesos.

### Does this PR introduce _any_ user-facing change?
Yes, it adds a `spark.mesos.driver.memoryOverhead` configuration option.  Documentation changes for this option are included in the PR.

### How was this patch tested?
Test cases covering allocation of driver memory overhead are included in the changes.

### Other notes
This is a second attempt to get this change reviewed, accepted and merged.  The original pull request was closed as stale back in January: https://github.com/apache/spark/pull/21006.
For this pull request, I took the original change by pmackles, rebased it onto the current master branch, and added a test case that was requested in the original code review.
I'm happy to make any further edits or do anything needed so that this can be included in a future spark release.  I keep having to build custom spark distributions so that we can use spark within our mesos clusters.

Closes #30739 from dmcwhorter/dmcwhorter-SPARK-22256.

Lead-authored-by: David McWhorter <david_mcwhorter@premierinc.com>
Co-authored-by: Paul Mackles <pmackles@adobe.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-15 14:00:38 -08: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
Holden Karau 5885cc15ca [SPARK-33261][K8S] Add a developer API for custom feature steps
### What changes were proposed in this pull request?

Add a developer API for custom driver & executor feature steps.

### Why are the changes needed?

While we allow templates for the basis of pod creation, some deployments need more flexibility in how the pods are configured. This adds a developer API for custom deployments.

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

New developer API.

### How was this patch tested?

Extended tests to verify custom step is applied when configured.

Closes #30206 from holdenk/SPARK-33261-allow-people-to-extend-pod-feature-steps.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-12-14 12:05:28 -08:00
Holden Karau bf2c88ccae
[SPARK-33716][K8S] Fix potential race condition during pod termination
### What changes were proposed in this pull request?

Check that the pod state is not pending or running even if there is a deletion timestamp.

### Why are the changes needed?

This can occur when the pod state and deletion timestamp are not updated by etcd in sync & we get a pod snapshot during an inconsistent view.

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

No

### How was this patch tested?

Manual testing with local version of Minikube on an overloaded computer that caused out of sync updates.

Closes #30693 from holdenk/SPARK-33716-decommissioning-race-condition-during-pod-snapshot.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-14 02:09:59 -08: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
Kousuke Saruta d662b95535 [SPARK-33754][K8S][DOCS] Update kubernetes/integration-tests/README.md to follow the default Hadoop profile updated
### What changes were proposed in this pull request?

This PR updates `kubernetes/integration-tests/README.md`.

### Why are the changes needed?

To follow the current Hadoop profile (hadoop-3.2).

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

No.

### How was this patch tested?

I have confirmed that the integration tests pass with the following command for both Hadoop 3.2 an 2.7.
```
build/mvn integration-test -am -pl :spark-kubernetes-integration-tests_2.12 \
  -Pkubernetes \
  -Pkubernetes-integration-tests \
  -Dspark.kubernetes.test.imageTag=${IMAGE_TAG} \
  -Dspark.kubernetes.test.imageRepo=docker.io/kubespark \
  -Dspark.kubernetes.test.namespace=default \
  -Dspark.kubernetes.test.deployMode=minikube \
  -Dtest.include.tags=k8s
```

Closes #30726 from sarutak/update-kube-integ-readme.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-11 01:52:13 -08:00
Kousuke Saruta 795db05bf6
[SPARK-33732][K8S][TESTS] Kubernetes integration tests doesn't work with Minikube 1.9+
### What changes were proposed in this pull request?

This PR changes `Minikube.scala` for Kubernetes integration tests to work with Minikube 1.9+.
`Minikube.scala` assumes that `apiserver.key` and `apiserver.crt` are in `~/.minikube/`.
But as of Minikube 1.9, they are in `~/.minikube/profiles/<profile>`.

### Why are the changes needed?

Currently, Kubernetes integration tests doesn't work with Minikube 1.9+.

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

No.

### How was this patch tested?

I confirmed the following test passes.
```
$ build/sbt -Pkubernetes -Pkubernetes-integration-tests package 'kubernetes-integration-tests/testOnly -- -z "SparkPi with no"'
```

Closes #30700 from sarutak/minikube-1.9.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-09 22:04:09 -08:00
Holden Karau 1c7f5f1ac7
[SPARK-33724][K8S] Add decom script as a configuration param
### What changes were proposed in this pull request?

Makes the location of the decommission script used in Kubernetes for graceful shutdown configurable.

### Why are the changes needed?

Some environments don't use the Spark image builder and instead mount the decompressed Spark distro. In those envs configuring the location of the decommissioning script is required.

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

New configuration parameter.

### How was this patch tested?

Existing decommissioning integration test.

Closes #30694 from holdenk/SPARK-33724-allow-decommissioning-script-location-to-be-configured.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-09 20:42:10 -08:00
Holden Karau 991b7977b5 [SPARK-33727][K8S] Fall back from gnupg.net to openpgp.org
### What changes were proposed in this pull request?

While building R docker image if we can't fetch the key from gnupg.net fall back to openpgp.org

### Why are the changes needed?

gnupg.net key servers are flaky and sometimes fail to resolve or return keys.

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

No

### How was this patch tested?

Tried to add key on my desktop, it failed, then tried to add key with openpgp.org and it succeed.

Closes #30696 from holdenk/SPARK-33727-gnupg-server-is-flaky.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-10 11:35:55 +09:00
suqilong 48f93af9f3 [SPARK-33669] Wrong error message from YARN application state monitor when sc.stop in yarn client mode
### What changes were proposed in this pull request?
This change make InterruptedIOException to be treated as InterruptedException when closing YarnClientSchedulerBackend, which doesn't log error with "YARN application has exited unexpectedly xxx"

### Why are the changes needed?
For YarnClient mode, when stopping YarnClientSchedulerBackend, it first tries to interrupt Yarn application monitor thread. In MonitorThread.run() it catches InterruptedException to gracefully response to stopping request.

But client.monitorApplication method also throws InterruptedIOException when the hadoop rpc call is calling. In this case, MonitorThread will not know it is interrupted, a Yarn App failed is returned with "Failed to contact YARN for application xxxxx;  YARN application has exited unexpectedly with state xxxxx" is logged with error level. which confuse user a lot.

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

### How was this patch tested?
very simple patch, seems no need?

Closes #30617 from sqlwindspeaker/yarn-client-interrupt-monitor.

Authored-by: suqilong <suqilong@qiyi.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2020-12-09 01:21:13 -06:00
Prashant Sharma 6317ba29a1
[SPARK-33668][K8S][TEST] Fix flaky test "Verify logging configuration is picked from the provided
### What changes were proposed in this pull request?
Fix flaky test "Verify logging configuration is picked from the provided SPARK_CONF_DIR/log4j.properties."
The test is flaking, with multiple flaked instances - the reason for the failure has been similar to:

```

The code passed to eventually never returned normally. Attempted 109 times over 3.0079882413999997 minutes. Last failure message: Failure executing: GET at:
https://192.168.39.167:8443/api/v1/namespaces/b37fc72a991b49baa68a2eaaa1516463/pods/spark-pi-97a9bc76308e7fe3-exec-1/log?pretty=false. Message: pods "spark-pi-97a9bc76308e7fe3-exec-1" not found. Received status: Status(apiVersion=v1, code=404, details=StatusDetails(causes=[], group=null, kind=pods, name=spark-pi-97a9bc76308e7fe3-exec-1, retryAfterSeconds=null, uid=null, additionalProperties={}), kind=Status, message=pods "spark-pi-97a9bc76308e7fe3-exec-1" not found, metadata=ListMeta(_continue=null, remainingItemCount=null, resourceVersion=null, selfLink=null, additionalProperties={}), reason=NotFound, status=Failure, additionalProperties={}).. (KubernetesSuite.scala:402)

```
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36854/console
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36852/console
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36850/console
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36848/console
From the above failures, it seems, that executor finishes too quickly and is removed by spark before the test can complete.
So, in order to mitigate this situation, one way is to turn on the flag
   "spark.kubernetes.executor.deleteOnTermination"

### Why are the changes needed?

Fixes a flaky test.

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

No

### How was this patch tested?

Existing tests.
May be a few runs of jenkins integration test, may reveal if the problem is resolved or not.

Closes #30616 from ScrapCodes/SPARK-33668/fix-flaky-k8s-integration-test.

Authored-by: Prashant Sharma <prashsh1@in.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-05 23:04:55 -08: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
Prashant Sharma 91182d6cce
[SPARK-33626][K8S][TEST] Allow k8s integration tests to assert both driver and executor logs for expected log(s)
### What changes were proposed in this pull request?

Allow k8s integration tests to assert both driver and executor logs for expected log(s)

### Why are the changes needed?

Some of the tests will be able to provide full coverage of the use case, by asserting both driver and executor logs.

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

No

### How was this patch tested?

TBD

Closes #30568 from ScrapCodes/expectedDriverLogChanges.

Authored-by: Prashant Sharma <prashsh1@in.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-02 08:43:30 -08: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
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
Erik Krogen f3c2583cc3 [SPARK-33185][YARN][FOLLOW-ON] Leverage RM's RPC API instead of REST to fetch driver log links in yarn.Client
### What changes were proposed in this pull request?
This is a follow-on to PR #30096 which initially added support for printing direct links to the driver stdout/stderr logs from the application report output in `yarn.Client` using the `spark.yarn.includeDriverLogsLink` configuration. That PR made use of the ResourceManager's REST APIs to fetch the necessary information to construct the links. This PR proposes removing the dependency on the REST API, since the new logic is the only place in `yarn.Client` which makes use of this API, and instead leverages the RPC API via `YarnClient`, which brings the code in line with the rest of `yarn.Client`.

### Why are the changes needed?

While the old logic worked okay when running a Spark application in a "standard" environment with full access to Kerberos credentials, it can fail when run in an environment with restricted Kerberos credentials. In our case, this environment is represented by [Azkaban](https://azkaban.github.io/), but it likely affects other job scheduling systems as well. In such an environment, the application has delegation tokens which enabled it to communicate with services such as YARN, but the RM REST API is not typically covered by such delegation tokens (note that although YARN does actually support accessing the RM REST API via a delegation token as documented [here](https://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html#Cluster_Delegation_Tokens_API), it is a new feature in alpha phase, and most deployments are likely not retrieving this token today).

Besides this enhancement, leveraging the `YarnClient` APIs greatly simplifies the processing logic, such as removing all JSON parsing.

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

Very minimal user-facing changes on top of PR #30096. Basically expands the scope of environments in which that feature will operate correctly.

### How was this patch tested?

In addition to redoing the `spark-submit` testing as mentioned in PR #30096, I also tested this logic in a restricted-credentials environment (Azkaban). It succeeds where the previous logic would fail with a 401 error.

Closes #30450 from xkrogen/xkrogen-SPARK-33185-driverlogs-followon.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2020-11-30 14:40:51 -06:00
Josh Soref 13fd272cd3 Spelling r common dev mlib external project streaming resource managers python
### What changes were proposed in this pull request?

This PR intends to fix typos in the sub-modules:
* `R`
* `common`
* `dev`
* `mlib`
* `external`
* `project`
* `streaming`
* `resource-managers`
* `python`

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 #30402 from jsoref/spelling-R_common_dev_mlib_external_project_streaming_resource-managers_python.

Authored-by: Josh Soref <jsoref@users.noreply.github.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-11-27 10:22:45 -06:00
Dongjoon Hyun 3ce4ab545b
[SPARK-33513][BUILD] Upgrade to Scala 2.13.4 to improve exhaustivity
### 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>
2020-11-23 16:28:43 -08:00
Venkata krishnan Sowrirajan 8218b48803 [SPARK-32919][SHUFFLE][TEST-MAVEN][TEST-HADOOP2.7] Driver side changes for coordinating push based shuffle by selecting external shuffle services for merging partitions
### 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>
2020-11-20 06:00:30 -06:00
yangjie01 e3058ba17c [SPARK-33441][BUILD] Add unused-imports compilation check and remove all unused-imports
### 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>
2020-11-19 14:20:39 +09:00
Stavros Kontopoulos dcac78e12b
[SPARK-27936][K8S] Support python deps
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>
2020-11-18 10:43:41 -08:00
Rameshkrishnan Muthusamy 5e8549973d
[SPARK-33471][K8S][BUILD] Upgrade kubernetes-client to 4.12.0
### What changes were proposed in this pull request?

This PR aims to upgrade Kubernetes-client from 4.11.1 to 4.12.0

### Why are the changes needed?

This upgrades the dependency for Apache Spark 3.1.0.

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

No.

### How was this patch tested?

Pass the CIs.

Closes #30401 from ramesh-muthusamy/SPARK-33471-k8s-clientupgrade.

Authored-by: Rameshkrishnan Muthusamy <rameshkrishnan_muthusamy@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-17 13:41:58 -08:00
Prashant Sharma 2a8e253cdb
[SPARK-32222][K8S][TESTS] Add K8s IT for conf propagation
### What changes were proposed in this pull request?

Added integration test - which tries to configure a log4j.properties and checks if, it is the one pickup by the driver.

### Why are the changes needed?

Improved test coverage.

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

No

### How was this patch tested?

By running integration tests.

Closes #30388 from ScrapCodes/SPARK-32222/k8s-it-spark-conf-propagate.

Authored-by: Prashant Sharma <prashsh1@in.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-17 08:47:04 -08:00
Pascal Gillet 9ab0f82a59
[SPARK-23499][MESOS] Support for priority queues in Mesos scheduler
### What changes were proposed in this pull request?

I push this PR as I could not re-open the stale one https://github.com/apache/spark/pull/20665 .

As for Yarn or Kubernetes, Mesos users should be able to specify priority queues to define a workload management policy for queued drivers in the Mesos Cluster Dispatcher.

This would ensure scheduling order while enqueuing Spark applications for a Mesos cluster.

### Why are the changes needed?

Currently, submitted drivers are kept in order of their submission: the first driver added to the queue will be the first one to be executed (FIFO), regardless of their priority.

See https://issues.apache.org/jira/projects/SPARK/issues/SPARK-23499 for more details.

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

The MesosClusterDispatcher UI shows now Spark jobs along with the queue to which they are submitted.

### How was this patch tested?

Unit tests.
Also, this feature has been in production for 3 years now as we use a modified Spark 2.4.0 since then.

Closes #30352 from pgillet/mesos-scheduler-priority-queue.

Lead-authored-by: Pascal Gillet <pascal.gillet@stack-labs.com>
Co-authored-by: pgillet <pascalgillet@ymail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-16 16:54:08 -08:00
Prashant Sharma 8615f354a4
[SPARK-30985][K8S] Support propagating SPARK_CONF_DIR files to driver and executor pods
### What changes were proposed in this pull request?
This is an improvement, we mount all the user specific configuration files(except the templates and spark properties files) from `SPARK_CONF_DIR` at the point of spark-submit, to both executor and driver pods. Currently, only `spark.properties` is mounted, only on driver.

### Why are the changes needed?

`SPARK_CONF_DIR` hosts several configuration files, for example,
1) `spark-defaults.conf` - containing all the spark properties.
2) `log4j.properties` - Logger configuration.
3) `core-site.xml` - Hadoop related configuration.
4) `fairscheduler.xml` - Spark's fair scheduling policy at the job level.
5) `metrics.properties` - Spark metrics.
6) Any user specific - library or framework specific configuration file.

At the moment, we can cannot propagate these files to the driver and executor configuration directory.

There is a design doc, with more details, and this patch is currently providing a reference implementation. Please take a look at the doc and comment, how we can improve. [google docs link to the doc](https://bit.ly/spark-30985)

### Further scope
Support user defined configMaps.

### Does this PR introduce any user-facing change?
Yes, previously the user configuration files(e.g. hdfs-site.xml, log4j.properties etc...) were not propagated by default, now after this patch it is propagated to driver and executor pods' `SPARK_CONF_DIR`.

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

Also manually tested, by deploying it to a minikube cluster and observing the additional configuration files were present, and taking effect. For example, changes to log4j.properties was properly applied to executors.

Closes #27735 from ScrapCodes/SPARK-30985/spark-conf-k8s-propagate.

Authored-by: Prashant Sharma <prashsh1@in.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-16 00:02:18 -08:00
Yuming Wang f660946ef2 [SPARK-33288][YARN][FOLLOW-UP][TEST-HADOOP2.7] Fix type mismatch error
### What changes were proposed in this pull request?

This pr fix type mismatch error:
```
[error] /home/jenkins/workspace/spark-master-test-sbt-hadoop-2.7-hive-2.3/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala:320:52: type mismatch;
[error]  found   : Long
[error]  required: Int
[error]         Resource.newInstance(resourcesWithDefaults.totalMemMiB, resourcesWithDefaults.cores)
[error]                                                    ^
[error] one error found
```

### Why are the changes needed?

Fix compile issue.

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

No.

### How was this patch tested?

Existing test.

Closes #30375 from wangyum/SPARK-33288.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2020-11-16 11:28:52 +08:00
Chandni Singh 423ba5a160 [SPARK-32916][SHUFFLE][TEST-MAVEN][TEST-HADOOP2.7] Remove the newly added YarnShuffleServiceSuite.java
### What changes were proposed in this pull request?
This is a follow-up fix for the failing tests in `YarnShuffleServiceSuite.java`. This java class was introduced in https://github.com/apache/spark/pull/30062. The tests in the class fail when run with hadoop-2.7 profile:
```
[ERROR] testCreateDefaultMergedShuffleFileManagerInstance(org.apache.spark.network.yarn.YarnShuffleServiceSuite)  Time elapsed: 0.627 s  <<< ERROR!
java.lang.NoClassDefFoundError: org/apache/commons/logging/LogFactory
	at org.apache.spark.network.yarn.YarnShuffleServiceSuite.testCreateDefaultMergedShuffleFileManagerInstance(YarnShuffleServiceSuite.java:37)
Caused by: java.lang.ClassNotFoundException: org.apache.commons.logging.LogFactory
	at org.apache.spark.network.yarn.YarnShuffleServiceSuite.testCreateDefaultMergedShuffleFileManagerInstance(YarnShuffleServiceSuite.java:37)

[ERROR] testCreateRemoteBlockPushResolverInstance(org.apache.spark.network.yarn.YarnShuffleServiceSuite)  Time elapsed: 0 s  <<< ERROR!
java.lang.NoClassDefFoundError: Could not initialize class org.apache.spark.network.yarn.YarnShuffleService
	at org.apache.spark.network.yarn.YarnShuffleServiceSuite.testCreateRemoteBlockPushResolverInstance(YarnShuffleServiceSuite.java:47)

[ERROR] testInvalidClassNameOfMergeManagerWillUseNoOpInstance(org.apache.spark.network.yarn.YarnShuffleServiceSuite)  Time elapsed: 0.001 s  <<< ERROR!
java.lang.NoClassDefFoundError: Could not initialize class org.apache.spark.network.yarn.YarnShuffleService
	at org.apache.spark.network.yarn.YarnShuffleServiceSuite.testInvalidClassNameOfMergeManagerWillUseNoOpInstance(YarnShuffleServiceSuite.java:57)
```
A test suit for `YarnShuffleService` did exist here:
`resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala`
I missed this when I created `common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java`. Moving all the new tests to the earlier test suite fixes the failures with hadoop-2.7 even though why this happened is not clear.

### Why are the changes needed?
The newly added tests are failing when run with hadoop profile 2.7

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

### How was this patch tested?
Ran the unit tests with the default profile as well as hadoop 2.7 profile.
`build/mvn test -Dtest=none -DwildcardSuites=org.apache.spark.network.yarn.YarnShuffleServiceSuite -Phadoop-2.7 -Pyarn`
```
Run starting. Expected test count is: 11
YarnShuffleServiceSuite:
- executor state kept across NM restart
- removed applications should not be in registered executor file
- shuffle service should be robust to corrupt registered executor file
- get correct recovery path
- moving recovery file from NM local dir to recovery path
- service throws error if cannot start
- recovery db should not be created if NM recovery is not enabled
- SPARK-31646: metrics should be registered into Node Manager's metrics system
- create default merged shuffle file manager instance
- create remote block push resolver instance
- invalid class name of merge manager will use noop instance
Run completed in 2 seconds, 572 milliseconds.
Total number of tests run: 11
Suites: completed 2, aborted 0
Tests: succeeded 11, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #30349 from otterc/SPARK-32916-followup.

Authored-by: Chandni Singh <singh.chandni@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-11-13 16:16:23 -06:00
Thomas Graves acfd846753 [SPARK-33288][SPARK-32661][K8S] Stage level scheduling support for Kubernetes
### 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>
2020-11-13 16:04:13 -06:00
Dongjoon Hyun 22baf05a9e [SPARK-33408][SPARK-32354][K8S][R] Use R 3.6.3 in K8s R image and re-enable RTestsSuite
### What changes were proposed in this pull request?

This PR aims to use R 3.6.3 in K8s R image and re-enable `RTestsSuite`.

### Why are the changes needed?

Jenkins Server is using `R 3.6.3`.
```
+ SPARK_HOME=/home/jenkins/workspace/SparkPullRequestBuilder-K8s
+ /usr/bin/R CMD check --as-cran --no-tests SparkR_3.1.0.tar.gz
* using log directory ‘/home/jenkins/workspace/SparkPullRequestBuilder-K8s/R/SparkR.Rcheck’
* using R version 3.6.3 (2020-02-29)
```

OpenJDK docker image is using `R 3.5.2 (2018-12-20)` which is old and currently `spark-3.0.1` fails to run SparkR.
```
$ cd spark-3.0.1-bin-hadoop3.2

$ bin/docker-image-tool.sh -R kubernetes/dockerfiles/spark/bindings/R/Dockerfile -n build
...
	 exit code: 1
	 termination reason: Error
...

$ bin/spark-submit --master k8s://https://192.168.64.49:8443 --deploy-mode cluster --conf spark.kubernetes.container.image=spark-r:latest local:///opt/spark/examples/src/main/r/dataframe.R

$ k logs dataframe-r-b1c14b75b0c09eeb-driver
...
+ exec /usr/bin/tini -s -- /opt/spark/bin/spark-submit --conf spark.driver.bindAddress=172.17.0.4 --deploy-mode client --properties-file /opt/spark/conf/spark.properties --class org.apache.spark.deploy.RRunner local:///opt/spark/examples/src/main/r/dataframe.R
20/11/10 06:03:58 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
log4j:WARN No appenders could be found for logger (io.netty.util.internal.logging.InternalLoggerFactory).
log4j:WARN Please initialize the log4j system properly.
log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig for more info.
Error: package or namespace load failed for ‘SparkR’ in rbind(info, getNamespaceInfo(env, "S3methods")):
 number of columns of matrices must match (see arg 2)
In addition: Warning message:
package ‘SparkR’ was built under R version 4.0.2
Execution halted
```

In addition, this PR aims to recover the test coverage.

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

No.

### How was this patch tested?

Pass K8S IT Jenkins job.

Closes #30130 from dongjoon-hyun/SPARK-32354.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-12 15:36:31 +09:00
yangjie01 02fd52cfbc [SPARK-33352][CORE][SQL][SS][MLLIB][AVRO][K8S] Fix procedure-like declaration compilation warnings in Scala 2.13
### 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>
2020-11-08 12:51:48 -06:00
Erik Krogen 324275ae83 [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr
### What changes were proposed in this pull request?
Currently when run in `cluster` mode on YARN, the Spark `yarn.Client` will print out the application report into the logs, to be easily viewed by users. For example:
```
INFO yarn.Client:
 	 client token: Token { kind: YARN_CLIENT_TOKEN, service:  }
 	 diagnostics: N/A
 	 ApplicationMaster host: X.X.X.X
 	 ApplicationMaster RPC port: 0
 	 queue: default
 	 start time: 1602782566027
 	 final status: UNDEFINED
 	 tracking URL: http://hostname:8888/proxy/application_<id>/
 	 user: xkrogen
```

I propose adding, alongside the application report, some additional lines like:
```
         Driver Logs (stdout): http://hostname:8042/node/containerlogs/container_<id>/xkrogen/stdout?start=-4096
         Driver Logs (stderr): http://hostname:8042/node/containerlogs/container_<id>/xkrogen/stderr?start=-4096
```

This information isn't contained in the `ApplicationReport`, so it's necessary to query the ResourceManager REST API. For now I have added this as an always-on feature, but if there is any concern about adding this REST dependency, I think hiding this feature behind an off-by-default flag is reasonable.

### Why are the changes needed?
Typically, the tracking URL can be used to find the logs of the ApplicationMaster/driver while the application is running. Later, the Spark History Server can be used to track this information down, using the stdout/stderr links on the Executors page.

However, in the situation when the driver crashed _before_ writing out a history file, the SHS may not be aware of this application, and thus does not contain links to the driver logs. When this situation arises, it can be difficult for users to debug further, since they can't easily find their driver logs.

It is possible to reach the logs by using the `yarn logs` commands, but the average Spark user isn't aware of this and shouldn't have to be.

With this information readily available in the logs, users can quickly jump to their driver logs, even if it crashed before the SHS became aware of the application. This has the additional benefit of providing a quick way to access driver logs, which often contain useful information, in a single click (instead of navigating through the Spark UI).

### Does this PR introduce _any_ user-facing change?
Yes, some additional print statements will be created in the application report when using YARN in cluster mode.

### How was this patch tested?
Added unit tests for the parsing logic in `yarn.ClientSuite`. Also tested against a live cluster. When the driver is running:
```
INFO Client: Application report for application_XXXXXXXXX_YYYYYY (state: RUNNING)
INFO Client:
         client token: Token { kind: YARN_CLIENT_TOKEN, service:  }
         diagnostics: N/A
         ApplicationMaster host: host.example.com
         ApplicationMaster RPC port: ######
         queue: queue_name
         start time: 1604529046091
         final status: UNDEFINED
         tracking URL: http://host.example.com:8080/proxy/application_XXXXXXXXX_YYYYYY/
         user: xkrogen
         Driver Logs (stdout): http://host.example.com:8042/node/containerlogs/container_e07_XXXXXXXXX_YYYYYY_01_000001/xkrogen/stdout?start=-4096
         Driver Logs (stderr): http://host.example.com:8042/node/containerlogs/container_e07_XXXXXXXXX_YYYYYY_01_000001/xkrogen/stderr?start=-4096
INFO Client: Application report for application_XXXXXXXXX_YYYYYY (state: RUNNING)
```
I confirmed that when the driver has not yet launched, the report does not include the two Driver Logs items. Will omit the output here for brevity since it looks the same.

Closes #30096 from xkrogen/xkrogen-SPARK-33185-yarn-client-print.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2020-11-05 12:38:42 -06:00
Dongjoon Hyun 27d8136934 [SPARK-33324][K8S][BUILD] Upgrade kubernetes-client to 4.11.1
### What changes were proposed in this pull request?

This PR aims to upgrade `Kubernetes-client` from 4.10.3 to 4.11.1.

### Why are the changes needed?

This upgrades the dependency for Apache Spark 3.1.0.
Since 4.12.0 is still new and has a breaking API changes, this PR chooses the latest compatible one.

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

No.

### How was this patch tested?

Pass the all CIs including K8s IT.

Closes #30233 from dongjoon-hyun/SPARK-33324.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-11-02 22:23:26 -08:00
Thomas Graves 72ad9dcd5d [SPARK-32037][CORE] Rename blacklisting feature
### 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>
2020-10-30 17:16:53 -05:00
Holden Karau 491a0fb08b [SPARK-33262][K8S][FOLLOWUP] Verify pod allocation does not stall
### What changes were proposed in this pull request?

Add a test that pending executor does not stall pod allocation.

### Why are the changes needed?

Better test coverage

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

Test only change.

### How was this patch tested?

New test passes.

Closes #30205 from holdenk/verify-pod-allocation-does-not-stall.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-30 11:26:30 -07:00
Holden Karau 98f0a21991 [SPARK-33231][SPARK-33262][CORE] Make pod allocation executor timeouts configurable & allow scheduling with pending pods
### What changes were proposed in this pull request?

Make pod allocation executor timeouts configurable. Keep all known pods in mind when allocating executors to avoid over-allocating if the pending time is much higher than the allocation interval.

This PR increases the default wait time to 600s from the current 60s.

Since nodes can now remain "pending" for long periods of time, we allow additional batches to be scheduled during pending allocation but keep the total number of pods in account.

### Why are the changes needed?
The current executor timeouts do not match that of all real world clusters especially under load. While this can be worked around by increasing the allocation batch delay, that will decrease the speed at which the total number of executors will be able to be requested.

The increase in default timeout is needed to handle real-world testing environments I've encountered on moderately busy clusters and K8s clusters with their own underlying dynamic scale-up of hardware (e.g. GKE, EKS, etc.)

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

Yes new configuration property

### How was this patch tested?

Updated existing test to use the timeout from the new configuration property. Verified test failed without the update.

Closes #30155 from holdenk/SPARK-33231-make-pod-creation-timeout-configurable.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-27 11:54:08 -07:00
Dongjoon Hyun afa6aee4f5 [SPARK-33237][K8S][TESTS] Use default Hadoop-3.2 profile from K8s IT Jenkins job
### What changes were proposed in this pull request?

This PR aims to use `hadoop-3.2` profile in K8s IT Jenkins jobs.
- [x] Switch the default value of `HADOOP_PROFILE` from `hadoop-2.7` to `hadoop-3.2`.
- [x] Remove `-Phadoop2.7` from Jenkins K8s IT job.
    - https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/configure

**BEFORE**
```
./dev/make-distribution.sh --name ${DATE}-${REVISION} --r --pip --tgz -DzincPort=${ZINC_PORT} \
     -Phadoop-2.7 -Pkubernetes -Pkinesis-asl -Phive -Phive-thriftserver
```

**AFTER**
```
./dev/make-distribution.sh --name ${DATE}-${REVISION} --r --pip --tgz -DzincPort=${ZINC_PORT} \
     -Pkubernetes -Pkinesis-asl -Phive -Phive-thriftserver
```

### Why are the changes needed?

Since Apache Spark 3.1.0, Hadoop 3 is the default.

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

No.

### How was this patch tested?

Check the Jenkins K8s IT log and result.
- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34899/
```
+ /home/jenkins/workspace/SparkPullRequestBuilder-K8s/build/mvn clean package -DskipTests -DzincPort=4021 -Pkubernetes -Pkinesis-asl -Phive -Phive-thriftserver
Using `mvn` from path: /home/jenkins/tools/hudson.tasks.Maven_MavenInstallation/Maven_3.6.3/bin/mvn
[INFO] Scanning for projects...
[INFO] ------------------------------------------------------------------------
[INFO] Reactor Build Order:
[INFO]
```

Closes #30153 from dongjoon-hyun/SPARK-33237.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-26 15:29:12 -07:00
Shiqi Sun f659527727 [SPARK-30821][K8S] Handle executor failure with multiple containers
Handle executor failure with multiple containers

Added a spark property spark.kubernetes.executor.checkAllContainers,
with default being false. When it's true, the executor snapshot will
take all containers in the executor into consideration when deciding
whether the executor is in "Running" state, if the pod restart policy is
"Never". Also, added the new spark property to the doc.

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

Checking of all containers in the executor pod when reporting executor status, if the `spark.kubernetes.executor.checkAllContainers` property is set to true.

### Why are the changes needed?

Currently, a pod remains "running" as long as there is at least one running container. This prevents Spark from noticing when a container has failed in an executor pod with multiple containers. With this change, user can configure the behavior to be different. Namely, if any container in the executor pod has failed, either the executor process or one of its sidecars, the pod is considered to be failed, and it will be rescheduled.

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

Yes, new spark property added.
User is now able to choose whether to turn on this feature using the `spark.kubernetes.executor.checkAllContainers` property.

### How was this patch tested?

Unit test was added and all passed.
I tried to run integration test by following the instruction [here](https://spark.apache.org/developer-tools.html) (section "Testing K8S") and also [here](https://github.com/apache/spark/blob/master/resource-managers/kubernetes/integration-tests/README.md), but I wasn't able to run it smoothly as it fails to talk with minikube cluster. Maybe it's because my minikube version is too new (I'm using v1.13.1)...? Since I've been trying it for two days and still can't make it work, I decided to submit this PR and hopefully the Jenkins test will pass.

Closes #29924 from huskysun/exec-sidecar-failure.

Authored-by: Shiqi Sun <s.sun@salesforce.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-10-24 09:55:57 -07:00
HyukjinKwon 10bd42cd47 [SPARK-33104][BUILD] Exclude 'org.apache.hadoop:hadoop-yarn-server-resourcemanager:jar:tests'
### What changes were proposed in this pull request?

This PR proposes to exclude `org.apache.hadoop:hadoop-yarn-server-resourcemanager:jar:tests` from `hadoop-yarn-server-tests` when we use Hadoop 2 profile.

For some reasons, after SBT 1.3 upgrade at SPARK-21708, SBT starts to pull the dependencies of 'hadoop-yarn-server-tests'  with 'tests' classifier:

```
org/apache/hadoop/hadoop-common/2.7.4/hadoop-common-2.7.4-tests.jar
org/apache/hadoop/hadoop-yarn-common/2.7.4/hadoop-yarn-common-2.7.4-tests.jar
org/apache/hadoop/hadoop-yarn-server-resourcemanager/2.7.4/hadoop-yarn-server-resourcemanager-2.7.4-tests.jar
```
these were not pulled before the upgrade.

This specific `hadoop-yarn-server-resourcemanager-2.7.4-tests.jar` causes the problem (SPARK-33104)

1. When the test case creates the Hadoop configuration here,
    cc06266ade/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala (L122)

2. Such jars above have higher precedence in the class path, instead of the specified custom `core-site.xml` in the test:

    e93b8f02cd/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala (L1375)

3. Later, `core-site.xml` in the jar is picked instead in Hadoop's `Configuration`:

    Before this fix:

    ```
    jar:file:/.../https/maven-central.storage-download.googleapis.com/maven2/org/apache/hadoop/
    hadoop-yarn-server-resourcemanager/2.7.4/hadoop-yarn-server-resourcemanager-2.7.4-tests.jar!/core-site.xml
    ```

    After this fix:

    ```
    file:/.../spark/resource-managers/yarn/target/org.apache.spark.deploy.yarn.YarnClusterSuite/
    org.apache.spark.deploy.yarn.YarnClusterSuite-localDir-nm-0_0/
    usercache/.../filecache/10/__spark_conf__.zip/__hadoop_conf__/core-site.xml
    ```

4. the `core-site.xml` in the jar of course does not contain:

    2cfd215dc4/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala (L133-L141)

    and the specific test fails.

This PR uses some kind of hacky approach. It was excluded from  'hadoop-yarn-server-tests'  with 'tests' classifier, and then added back as a proper dependency (when Hadoop 2 profile is used). In this way, SBT does not pull `hadoop-yarn-server-resourcemanager` with `tests` classifier anymore.

### Why are the changes needed?

To make the build pass. This is a blocker.

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

No, test-only.

### How was this patch tested?

Manually tested and debugged:

```bash
build/sbt clean "yarn/testOnly *.YarnClusterSuite -- -z SparkHadoopUtil" -Pyarn -Phadoop-2.7 -Phive -Phive-2.3
```

Closes #30133 from HyukjinKwon/SPARK-33104.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-23 19:19:02 +09:00
yi.wu edeecada66 [SPARK-32850][CORE][K8S] Simplify the RPC message flow of decommission
### 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>
2020-10-23 13:58:44 +09:00
Chao Sun cb3fa6c936 [SPARK-33212][BUILD] Move to shaded clients for Hadoop 3.x profile
### 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>
2020-10-22 03:21:34 +00:00
HyukjinKwon 2cfd215dc4 [SPARK-33191][YARN][TESTS] Fix PySpark test cases in YarnClusterSuite
### 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>
2020-10-21 00:31:58 +09:00
Dongjoon Hyun 97605cd126 [SPARK-33175][K8S] Detect duplicated mountPaths and fail at Spark side
### What changes were proposed in this pull request?

This PR aims to detect duplicate `mountPath`s and stop the job.

### Why are the changes needed?

If there is a conflict on `mountPath`, the pod is created and repeats the following error messages and keeps running. Spark job should not keep running and wasting the cluster resources. We had better fail at Spark side.
```
$ k get pod -l 'spark-role in (driver,executor)'
NAME    READY   STATUS    RESTARTS   AGE
tpcds   1/1     Running   0          33m
```

```
20/10/18 05:09:26 WARN ExecutorPodsSnapshotsStoreImpl: Exception when notifying snapshot subscriber.
io.fabric8.kubernetes.client.KubernetesClientException: Failure executing: POST at: ...
Message: Pod "tpcds-exec-1" is invalid: spec.containers[0].volumeMounts[1].mountPath:
Invalid value: "/data1": must be unique.
...
```

**AFTER THIS PR**
The job will stop with the following error message instead of keeping running.
```
20/10/18 06:58:45 ERROR ExecutorPodsSnapshotsStoreImpl: Going to stop due to IllegalArgumentException
java.lang.IllegalArgumentException: requirement failed: Found duplicated mountPath: `/data1`
```

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

Yes, but this is a bug fix.

### How was this patch tested?

Pass the CI with the newly added test case.

Closes #30084 from dongjoon-hyun/SPARK-33175-2.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-18 09:59:50 -07:00
Dongjoon Hyun 20b7b923ab [SPARK-33176][K8S] Use 11-jre-slim as default in K8s Dockerfile
### What changes were proposed in this pull request?

This PR aims to use `openjdk:11-jre-slim` as default in K8s Dockerfile.

### Why are the changes needed?

Although Apache Spark supports both Java8/Java11, there is a difference.

1. Java8-built distribution can run both Java8/Java11
2. Java11-built distribution can run on Java11, but not Java8.

In short, we had better use Java11 in Dockerfile to embrace both cases without any issues.

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

Yes. This will remove the change of user frustration when they build with JDK11 and build the image without overriding Java base image.

### How was this patch tested?

Pass the K8s IT.

Closes #30083 from dongjoon-hyun/SPARK-33176.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-18 09:21:07 -07:00
Holden Karau ce6180c8c3 [SPARK-33154][CORE][K8S] Handle cleaned shuffles during migration
### 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>
2020-10-16 14:47:46 -07:00
Min Shen 82eea13c76 [SPARK-32915][CORE] Network-layer and shuffle RPC layer changes to support push shuffle blocks
### 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>
2020-10-15 12:34:52 -05:00
Dongjoon Hyun 8e7c39089f [SPARK-33155][K8S] spark.kubernetes.pyspark.pythonVersion allows only '3'
### What changes were proposed in this pull request?

This PR makes `spark.kubernetes.pyspark.pythonVersion` allow only `3`. In other words, it will reject `2` for `Python 2`.
- [x] Configuration description and check is updated.
- [x] Documentation is updated
- [x] Unit test cases are updated.
- [x] Docker image script is updated.

### Why are the changes needed?

After SPARK-32138, Apache Spark 3.1 dropped Python 2 support.

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

Yes, but Python 2 support is already dropped officially.

### How was this patch tested?

Pass the CI.

Closes #30049 from dongjoon-hyun/SPARK-DROP-PYTHON2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-15 01:51:01 -07:00
Dongjoon Hyun e1909c96fb [SPARK-33099][K8S] Respect executor idle timeout conf in ExecutorPodsAllocator
### What changes were proposed in this pull request?

This PR aims to protect the executor pod request or pending pod during executor idle timeout.

### Why are the changes needed?

In case of dynamic allocation, Apache Spark K8s `ExecutorPodsAllocator` cancels the pod requests or pending pods too eagerly. Like the following example, `ExecutorPodsAllocator` received the new total executor adjust request rapidly in two minutes. Sometimes, it's called 3 times in a single second. It repeats `request` and `delete` on that request or pending pod frequently. This PR is reusing `spark.dynamicAllocation.executorIdleTimeout (default: 60s)` to keep the pod request or pending pod.

```
20/10/08 05:58:08 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 3
20/10/08 05:58:08 INFO ExecutorPodsAllocator: Going to request 3 executors from Kubernetes.
20/10/08 05:58:09 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 3
20/10/08 05:58:43 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 1
20/10/08 05:58:47 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 0
20/10/08 05:59:26 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 3
20/10/08 05:59:30 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 2
20/10/08 05:59:31 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 3
20/10/08 05:59:44 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 2
20/10/08 05:59:44 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 0
20/10/08 05:59:45 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 3
20/10/08 05:59:50 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 2
20/10/08 05:59:50 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 1
20/10/08 05:59:50 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 0
20/10/08 05:59:54 INFO ExecutorPodsAllocator: Set totalExpectedExecutors to 3
20/10/08 05:59:54 INFO ExecutorPodsAllocator: Going to request 1 executors from Kubernetes.
```

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

No.

### How was this patch tested?

Pass the newly added test case.

Closes #29981 from dongjoon-hyun/SPARK-K8S-INITIAL.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-09 02:50:38 -07:00
Dongjoon Hyun 4987db8c88 [SPARK-33096][K8S] Use LinkedHashMap instead of Map for newlyCreatedExecutors
### What changes were proposed in this pull request?

This PR aims to use `LinkedHashMap` instead of `Map` for `newlyCreatedExecutors`.

### Why are the changes needed?

This makes log messages (INFO/DEBUG) more readable. This is helpful when `spark.kubernetes.allocation.batch.size` is large and especially when K8s dynamic allocation is used.

**BEFORE**
```
20/10/08 10:24:21 DEBUG ExecutorPodsAllocator: Executor with id 8 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:24:21 DEBUG ExecutorPodsAllocator: Executor with id 2 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:24:21 DEBUG ExecutorPodsAllocator: Executor with id 5 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:24:21 DEBUG ExecutorPodsAllocator: Executor with id 4 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:24:21 DEBUG ExecutorPodsAllocator: Executor with id 7 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:24:21 DEBUG ExecutorPodsAllocator: Executor with id 10 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:24:21 DEBUG ExecutorPodsAllocator: Executor with id 9 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:24:21 DEBUG ExecutorPodsAllocator: Executor with id 3 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:24:21 DEBUG ExecutorPodsAllocator: Executor with id 6 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:24:21 INFO ExecutorPodsAllocator: Deleting 9 excess pod requests (5,10,6,9,2,7,3,8,4).
```

**AFTER**
```
20/10/08 10:25:17 DEBUG ExecutorPodsAllocator: Executor with id 2 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:25:17 DEBUG ExecutorPodsAllocator: Executor with id 3 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:25:17 DEBUG ExecutorPodsAllocator: Executor with id 4 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:25:17 DEBUG ExecutorPodsAllocator: Executor with id 5 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:25:17 DEBUG ExecutorPodsAllocator: Executor with id 6 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:25:17 DEBUG ExecutorPodsAllocator: Executor with id 7 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:25:17 DEBUG ExecutorPodsAllocator: Executor with id 8 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:25:17 DEBUG ExecutorPodsAllocator: Executor with id 9 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:25:17 DEBUG ExecutorPodsAllocator: Executor with id 10 was not found in the Kubernetes cluster since it was created 0 milliseconds ago.
20/10/08 10:25:17 INFO ExecutorPodsAllocator: Deleting 9 excess pod requests (2,3,4,5,6,7,8,9,10).
```

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

No.

### How was this patch tested?

Pass the CI or `build/sbt -Pkubernetes "kubernetes/test"`

Closes #29979 from dongjoon-hyun/SPARK-K8S-LOG.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-08 11:50:53 -07:00
Stijn De Haes 3099fd9f9d [SPARK-32067][K8S] Use unique ConfigMap name for executor pod template
### What changes were proposed in this pull request?

The pod template configmap always had the same name. This PR makes it unique.

### Why are the changes needed?

If you scheduled 2 spark jobs they will both use the same configmap name this will result in conflicts. This PR fixes that

**BEFORE**
```
$ kubectl get cm --all-namespaces -w | grep podspec
podspec-configmap                              1      65s
```

**AFTER**
```
$ kubectl get cm --all-namespaces -w | grep podspec
aaece65ef82e4a30b7b7800aad600d4f   spark-test-app-aac9f37502b2ca55-driver-podspec-conf-map   1      0s
```

This can be seen when running the integration tests

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

No

### How was this patch tested?

Unit tests and the integration tests test if this works

Closes #29934 from stijndehaes/bugfix/SPARK-32067-unique-name-for-template-configmap.

Authored-by: Stijn De Haes <stijndehaes@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-07 09:52:00 -07:00
gschiavon a09747bf32 [SPARK-33063][K8S] Improve error message for insufficient K8s volume confs
### What changes were proposed in this pull request?
Provide error handling when creating kubernetes volumes. Right now they keys are expected to be there and if not it fails with a `key not found` error, but not knowing why do you need that `key`.

Also I renamed some tests that didn't indicate the kind of kubernetes volume

### Why are the changes needed?

Easier for the users to understand why `spark-submit` command is failing if not providing they right kubernetes volumes properties.

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

### How was this patch tested?
It was tested with the current tests plus added one more.

[Jira ticket](https://issues.apache.org/jira/browse/SPARK-33063)

Closes #29941 from Gschiavon/SPARK-33063-provide-error-handling-k8s-volumes.

Authored-by: gschiavon <germanschiavon@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-05 09:02:06 -07:00
Kousuke Saruta fab53212cb [SPARK-33065][TESTS] Expand the stack size of a thread in a test in LocalityPlacementStrategySuite for Java 11 with sbt
### What changes were proposed in this pull request?

This PR fixes an issue that a test in `LocalityPlacementStrategySuite` fails with Java 11 due to `StackOverflowError`.

```
[info] - handle large number of containers and tasks (SPARK-18750) *** FAILED *** (170 milliseconds)
[info]   StackOverflowError should not be thrown; however, got:
[info]
[info]   java.lang.StackOverflowError
[info]          at java.base/java.util.concurrent.ConcurrentHashMap.putVal(ConcurrentHashMap.java:1012)
[info]          at java.base/java.util.concurrent.ConcurrentHashMap.putIfAbsent(ConcurrentHashMap.java:1541)
[info]          at java.base/java.lang.ClassLoader.getClassLoadingLock(ClassLoader.java:668)
[info]          at java.base/jdk.internal.loader.BuiltinClassLoader.loadClassOrNull(BuiltinClassLoader.java:591)
[info]          at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:579)
[info]          at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178)
[info]          at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:522)
```

The solution is to expand the stack size of a thread in the test from 32KB to 256KB.
Currently, the stack size is specified as 32KB but the actual stack size can be greater than 32KB.
According to the code of Hotspot, the minimum stack size is prefer to the specified size.

Java 8: https://hg.openjdk.java.net/jdk8u/jdk8u/hotspot/file/c92ba514724d/src/os/linux/vm/os_linux.cpp#l900
Java 11: https://hg.openjdk.java.net/jdk-updates/jdk11u/file/73edf743a93a/src/hotspot/os/posix/os_posix.cpp#l1555

For Linux on x86_64, the minimum stack size seems to be 224KB and 136KB for Java 8 and Java 11 respectively. So, the actual stack size should be 224KB rather than 32KB for Java 8 on x86_64/Linux.
As the test passes for Java 8 but doesn't for Java 11, 224KB is enough while 136KB is not.
So I think specifing 256KB is reasonable for the new stack size.

### Why are the changes needed?

To pass the test for Java 11.

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

No.

### How was this patch tested?

Following command with Java 11.
```
build/sbt -Pyarn clean package "testOnly org.apache.spark.deploy.yarn.LocalityPlacementStrategySuite"
```

Closes #29943 from sarutak/fix-stack-size.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-04 16:11:06 -07:00
jlafleche d75222dd1b [SPARK-33012][BUILD][K8S] Upgrade fabric8 to 4.10.3
### What changes were proposed in this pull request?

This PR aims to upgrade `kubernetes-client` library to track fabric8's declared compatibility for k8s 1.18.0:
https://github.com/fabric8io/kubernetes-client#compatibility-matrix

### Why are the changes needed?
According to fabric8, 4.9.2 is incompatible with k8s 1.18.0.

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

No.

### How was this patch tested?

Not tested yet.

Closes #29888 from laflechejonathan/jlf/fabric8Ugprade.

Authored-by: jlafleche <jlafleche@palantir.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-30 19:00:18 -07:00
Dongjoon Hyun 6c805470a7 [SPARK-32997][K8S] Support dynamic PVC creation and deletion in K8s driver
### What changes were proposed in this pull request?

This PR aims to support dynamic PVC creation and deletion in K8s driver.

**Configuration**
This PR reuses the existing PVC volume configs.
```
spark.kubernetes.driver.volumes.persistentVolumeClaim.spark-local-dir-1.options.claimName=OnDemand
spark.kubernetes.driver.volumes.persistentVolumeClaim.spark-local-dir-1.options.storageClass=gp2
spark.kubernetes.driver.volumes.persistentVolumeClaim.spark-local-dir-1.options.sizeLimit=200Gi
spark.kubernetes.driver.volumes.persistentVolumeClaim.spark-local-dir-1.mount.path=/data
spark.kubernetes.driver.volumes.persistentVolumeClaim.spark-local-dir-1.mount.readOnly=false
```

**PVC**
```
$ kubectl get pvc | grep driver
tpcds-d6087874c6705564-driver-pvc-0  Bound    pvc-fae914a2-ca5c-4e1e-8aba-54a35357d072   200Gi RWO gp2 12m
```

**Disk**
```
$ k exec -it tpcds-d6087874c6705564-driver -- df -h | grep data
/dev/nvme5n1    197G   61M  197G   1% /data
```

```
$ k exec -it tpcds-d6087874c6705564-driver -- ls -al /data
total 28
drwxr-xr-x  5 root root  4096 Sep 25 18:06 .
drwxr-xr-x  1 root root    63 Sep 25 18:06 ..
drwxr-xr-x 66 root root  4096 Sep 25 18:09 blockmgr-2c9a8cc5-a05c-45fe-a58e-b8f42da88a57
drwx------  2 root root 16384 Sep 25 18:06 lost+found
drwx------  4 root root  4096 Sep 25 18:07 spark-0448efe7-da2c-4f3a-bd3c-769aadb11dd6
```

**NOTE**
This should be used carefully because Apache Spark doesn't delete driver pod automatically. Since the driver PVC shares the lifecycle of driver pod, it will exist after the job completion until the pod deletion. However, if the users are already using pre-populated PVCs, this isn't a regression at all in terms of the cost.

```
$ k get pod -l spark-role=driver
NAME                            READY   STATUS      RESTARTS   AGE
tpcds-d6087874c6705564-driver   0/1     Completed   0          35m
```

### Why are the changes needed?

Like executors, driver also needs larger PVC.

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

Yes. This is a new feature.

### How was this patch tested?

Pass the newly added test case.

Closes #29873 from dongjoon-hyun/SPARK-32997.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-25 16:36:15 -07:00
yangjie01 4ae0f70395 [SPARK-32954][YARN][TEST] Add jakarta.servlet-api test dependency to yarn module to avoid UTs badcase
### What changes were proposed in this pull request?

When I tried to verify that the `resource-managers/yarn` module passed all UTs in Scala 2.13 , I found that there is a
issue related to classpath order maybe blocked the UTs because there are more than one `servlet-api` dependency in spark now:

- One is `javax.servlet:javax.servlet-api:3.10:compile` config in core/pom.xml,

- The other is `jakarta.servlet:jakarta.servlet-api:4.0.3:test`  cascaded by `org.glassfish.jersey.test-framework.providers`

we can use `mvn dependency:tree` to check it .

So when we execute `resource-managers/yarn` module test use

```
mvn clean test -pl resource-managers/yarn -Pyarn
```
or
```
mvn clean test -pl resource-managers/yarn -Pyarn -Pscala-2.13
```

and if the position of `javax.servlet-api` in the  in classpath is before `jakarta.servlet-api`, there are some cases failed in `YarnClusterSuite`, `YarnShuffleIntegrationSuite`  and `YarnShuffleAuthSuite`.

The failed reason as follow:

```
20/09/18 19:14:07.486 launcher-proc-1 INFO YarnClusterDriver: Exception in thread "main" java.lang.ExceptionInInitializerError
...
20/09/18 19:14:07.486 launcher-proc-1 INFO YarnClusterDriver: Caused by: java.lang.SecurityException: class "javax.servlet.http.HttpSessionIdListener"'s signer information does not match signer information of other classes in the same package
...
```

### Why are the changes needed?

Avoid UTs error caused by classpath order .

### 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: Pass 2.13 Build GitHub Action and do the following:

```
dev/change-scala-version.sh 2.13
mvn clean install -DskipTests -pl resource-managers/yarn -Pyarn -Pscala-2.13 -am
mvn clean test -pl resource-managers/yarn -Pyarn -Pscala-2.13
```

```
Tests: succeeded 136, failed 0, canceled 1, ignored 0, pending 0
All tests passed.
```

Closes #29824 from LuciferYang/yarn-tests-deps.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-24 08:32:32 -07:00
yangjie01 fe6d38d243 [SPARK-32987][MESOS] Pass all mesos module UTs in Scala 2.13
### What changes were proposed in this pull request?
The main change of this pr is add a manual sort to `defaultConf ++ driverConf` before constructing `--conf` options to ensure options has same order in Scala 2.12 and Scala 2.13.

### 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: Pass GitHub 2.13 Build Action

Do the following:

```
dev/change-scala-version.sh 2.13
mvn clean install -DskipTests  -pl resource-managers/mesos -Pscala-2.13 -Pmesos -am
mvn test -pl resource-managers/mesos -Pscala-2.13 -Pmesos
```

**Before**
```
Tests: succeeded 106, failed 1, canceled 0, ignored 0, pending 0
*** 1 TESTS FAILED ***
```

**After**

```
Tests: succeeded 107, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #29865 from LuciferYang/SPARK-32987-2.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-24 08:25:24 -07:00
Dongjoon Hyun 0bc0e91e40 [SPARK-32971][K8S][FOLLOWUP] Add .toSeq for Scala 2.13 compilation
### What changes were proposed in this pull request?

This is a follow-up to fix Scala 2.13 compilation at Kubernetes module.

### Why are the changes needed?

To fix Scala 2.13 compilation.

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

No.

### How was this patch tested?

Pass the GitHub Action Scala 2.13 compilation job.

Closes #29859 from dongjoon-hyun/SPARK-32971-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-23 20:10:01 -07:00
Dongjoon Hyun 527cd3fc3a [SPARK-32971][K8S] Support dynamic PVC creation/deletion for K8s executors
### What changes were proposed in this pull request?

This PR aims to support dynamic PVC creation and deletion for K8s executors. The PVCs are created with executor pods and deleted when the executor pods are deleted.

**Configuration**
Mostly, this PR reuses the existing PVC volume configs and `storageClass` is added.
```
spark.executor.instances=2
spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.options.claimName=OnDemand
spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.options.storageClass=gp2
spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.options.sizeLimit=500Gi
spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount.path=/data
spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount.readOnly=false
```

**Executors**
```
$ kubectl get pod -l spark-role=executor
NAME                               READY   STATUS    RESTARTS   AGE
spark-pi-f4d80574b9bb0941-exec-1   1/1     Running   0          2m6s
spark-pi-f4d80574b9bb0941-exec-2   1/1     Running   0          2m6s
```

**PVCs**
```
$ kubectl get pvc
NAME                                     STATUS   VOLUME                                     CAPACITY   ACCESS MODES   STORAGECLA
SS   AGE
spark-pi-f4d80574b9bb0941-exec-1-pvc-0   Bound    pvc-7d20173f-278b-4c7b-b7e5-7f0ed414ee64   500Gi      RWO            gp2
     48s
spark-pi-f4d80574b9bb0941-exec-2-pvc-0   Bound    pvc-1138f00d-87f1-47f4-9b58-ce5d13ea0c3a   500Gi      RWO            gp2
     48s
```

**Executor Disk**
```
$ k exec -it spark-pi-f4d80574b9bb0941-exec-1 -- df -h /data
Filesystem      Size  Used Avail Use% Mounted on
/dev/nvme3n1    493G   74M  492G   1% /data
```

```
$ k exec -it spark-pi-f4d80574b9bb0941-exec-1 -- ls /data
blockmgr-81dcebaf-11a7-4d7b-91d6-3c580187d914
lost+found
spark-6be42db8-2c58-4389-b52c-8aeeafe76bd5
```

### Why are the changes needed?

While SPARK-32655 supports to mount a pre-created PVC, this PR can create PVC itself dynamically and reduce lots of manual efforts.

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

Yes. This is a new feature.

### How was this patch tested?

Pass the newly added test cases.

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

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-23 16:47:10 -07:00
Holden Karau 27f6b5a103 [SPARK-32937][SPARK-32980][K8S] Fix decom & launcher tests and add some comments to reduce chance of breakage
### 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>
2020-09-23 15:39:31 -07:00
Kent Yao 9e9d4b6994 [SPARK-32905][CORE][YARN] ApplicationMaster fails to receive UpdateDelegationTokens message
### What changes were proposed in this pull request?

With a long-running application in kerberized mode, the AMEndpiont handles `UpdateDelegationTokens` message wrong, which is an OneWayMessage that should be handled in the `receive` function.

```java
20-09-15 18:53:01 INFO yarn.YarnAllocator: Received 22 containers from YARN, launching executors on 0 of them.
20-09-16 12:52:28 ERROR netty.Inbox: Ignoring error
org.apache.spark.SparkException: NettyRpcEndpointRef(spark-client://YarnAM) does not implement 'receive'
	at org.apache.spark.rpc.RpcEndpoint$$anonfun$receive$1.applyOrElse(RpcEndpoint.scala:70)
	at org.apache.spark.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115)
	at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:203)
	at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:100)
	at org.apache.spark.rpc.netty.MessageLoop.org$apache$spark$rpc$netty$MessageLoop$$receiveLoop(MessageLoop.scala:75)
	at org.apache.spark.rpc.netty.MessageLoop$$anon$1.run(MessageLoop.scala:41)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
20-09-17 06:52:28 ERROR netty.Inbox: Ignoring error
org.apache.spark.SparkException: NettyRpcEndpointRef(spark-client://YarnAM) does not implement 'receive'
	at org.apache.spark.rpc.RpcEndpoint$$anonfun$receive$1.applyOrElse(RpcEndpoint.scala:70)
	at org.apache.spark.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115)
	at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:203)
	at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:100)
	at org.apache.spark.rpc.netty.MessageLoop.org$apache$spark$rpc$netty$MessageLoop$$receiveLoop(MessageLoop.scala:75)
	at org.apache.spark.rpc.netty.MessageLoop$$anon$1.run(MessageLoop.scala:41)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

### Why are the changes needed?

bugfix, without a proper token refresher, the long-running apps are going to fail potentially in kerberized cluster

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

no

### How was this patch tested?

Passing jenkins

and verify manually

I am running the sub-module `kyuubi-spark-sql-engine` of https://github.com/yaooqinn/kyuubi

The simplest way to reproduce the bug and verify this fix is to follow these steps

#### 1 build the `kyuubi-spark-sql-engine` module
```
mvn clean package -pl :kyuubi-spark-sql-engine
```
#### 2. config the spark with Kerberos settings towards your secured cluster

#### 3. start it in the background
```
nohup bin/spark-submit --class org.apache.kyuubi.engine.spark.SparkSQLEngine ../kyuubi-spark-sql-engine-1.0.0-SNAPSHOT.jar > kyuubi.log &
```

#### 4. check the AM log and see

"Updating delegation tokens ..." for SUCCESS

"Inbox: Ignoring error ...... does not implement 'receive'" for FAILURE

Closes #29777 from yaooqinn/SPARK-32905.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-18 07:41:21 +00:00
William Hyun d58a4a310a [SPARK-32882][K8S] Remove python2 installation in K8s python image
### What changes were proposed in this pull request?
This PR aims to remove python2 installation in K8s python image because spark 3.1 does not support python2.

### Why are the changes needed?

This will save disk space.

**BEFORE**
```
kubespark/spark-py ... 917MB
```

**AFTER**
```
kubespark/spark-py ... 823MB
```

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

### How was this patch tested?

Pass the Jenkins with the K8s IT.

Closes #29751 from williamhyun/remove_py2.

Authored-by: William Hyun <williamhyun3@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-14 16:03:19 -07:00
Dongjoon Hyun 182727d90f [SPARK-32713][K8S] Support execId placeholder in executor PVC conf
### What changes were proposed in this pull request?

This PR aims to support executor id placeholder in `spark.kubernetes.executor.volumes.persistentVolumeClaim.myname.options.claimName` configuration like the following.
```
--conf spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.options.claimName=pvc-spark-SPARK_EXECUTOR_ID \
```

### Why are the changes needed?

This is a convenient way to mount corresponding PV to the executor.

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

Yes, but this is a new feature and there is no regression because users don't use `SPARK_EXECUTOR_ID` in PVC claim name.

### How was this patch tested?

Pass the newly added test case.

Closes #29557 from dongjoon-hyun/SPARK-PVC.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-27 09:49:21 -07:00
farhan5900 8749f2e87a [SPARK-32675][MESOS] --py-files option is appended without passing value for it
### What changes were proposed in this pull request?
The PR checks for the emptiness of `--py-files` value and uses it only if it is not empty.

### Why are the changes needed?
There is a bug in Mesos cluster mode REST Submission API. It is using `--py-files` option without specifying any value for the conf `spark.submit.pyFiles` by the user.

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

### How was this patch tested?
* Submitting an application to a Mesos cluster:
`curl -X POST http://localhost:7077/v1/submissions/create --header "Content-Type:application/json" --data '{
"action": "CreateSubmissionRequest",
"appResource": "file:///opt/spark-3.0.0-bin-3.2.0/examples/jars/spark-examples_2.12-3.0.0.jar",
"clientSparkVersion": "3.0.0",
"appArgs": ["30"],
"environmentVariables": {},
"mainClass": "org.apache.spark.examples.SparkPi",
"sparkProperties": {
  "spark.jars": "file:///opt/spark-3.0.0-bin-3.2.0/examples/jars/spark-examples_2.12-3.0.0.jar",
  "spark.driver.supervise": "false",
  "spark.executor.memory": "512m",
  "spark.driver.memory": "512m",
  "spark.submit.deployMode": "cluster",
  "spark.app.name": "SparkPi",
  "spark.master": "mesos://localhost:5050"
}}'`
* It should be able to pick the correct class and run the job successfully.

Closes #29499 from farhan5900/SPARK-32675.

Authored-by: farhan5900 <farhan5900@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-23 17:24:10 -07:00
Holden Karau 059fb6571e [SPARK-32657][K8S] Update the log strings we check for & imports in decommission K8s
### What changes were proposed in this pull request?

Update the log strings to match the new log messages.

### Why are the changes needed?

Tests are failing

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

No, test only change.

### How was this patch tested?
WIP: Make sure the DecommissionSuite passes in Jenkins.

Closes #29479 from holdenk/SPARK-32657-Decommissioning-tests-update-log-string.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-19 18:28:21 -07:00
Dongjoon Hyun 3722ed430d [SPARK-32655][K8S] Support appId/execId placeholder in K8s SPARK_EXECUTOR_DIRS
### What changes were proposed in this pull request?

This PR aims to support replacements of `SPARK_APPLICATION_ID`/`SPARK_EXECUTOR_ID` in `SPARK_EXECUTOR_DIRS ` executor environment.

### Why are the changes needed?

This PR provides users additional controllability.

**HOW TO RUN**
```
bin/spark-submit --master k8s://https://kubernetes.docker.internal:6443 --deploy-mode cluster \
-c spark.kubernetes.container.image=spark:SPARK-32655 \
-c spark.kubernetes.driver.pod.name=pi \
-c spark.kubernetes.executor.podNamePrefix=pi \
-c spark.kubernetes.executor.volumes.nfs.data.mount.path=/efs \
-c spark.kubernetes.executor.volumes.nfs.data.mount.readOnly=false \
-c spark.kubernetes.executor.volumes.nfs.data.options.server=efs-server-ip \
-c spark.kubernetes.executor.volumes.nfs.data.options.path=/ \
-c spark.executorEnv.SPARK_EXECUTOR_DIRS=/efs/SPARK_APPLICATION_ID/SPARK_EXECUTOR_ID \
--class org.apache.spark.examples.SparkPi \
local:///opt/spark/examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar 20000
```

**EFS Layout**
```
/efs
├── spark-f45039b13b0b4fd4baf80fed561a2228
│   ├── 1
│   │   ├── blockmgr-bbe76578-8ff2-4c2d-ab4f-37671d886f56
│   │   │   ├── 0e
│   │   │   └── 11
│   │   └── spark-e41aeb41-00fc-49e1-a77d-093b6df5958a
│   │       ├── -18375678081597852666997_cache
│   │       └── -18375678081597852666997_lock
│   └── 2
│       ├── blockmgr-765bfb50-ab13-4b2b-9350-356fed0169e3
│       │   ├── 0e
│       │   └── 11
│       └── spark-737671fc-1697-4367-9daf-2b1575f92aba
│           ├── -18375678081597852666997_cache
│           └── -18375678081597852666997_lock
```

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

- Yes because this is a new feature.
- This will not affect the existing jobs because users don't use the string pattern `SPARK_APPLICATION_ID` or `SPARK_EXECUTOR_ID` inside `SPARK_EXECUTOR_DIRS` environment variable.

### How was this patch tested?

Pass the newly added test case.

Closes #29472 from dongjoon-hyun/SPARK-32655.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-19 12:11:34 -07:00
yi.wu 3092527f75 [SPARK-32651][CORE] Decommission switch configuration should have the highest hierarchy
### 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>
2020-08-19 06:53:06 +00:00
Holden Karau 548ac7c4af [SPARK-31198][CORE] Use graceful decommissioning as part of dynamic scaling
### 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>
2020-08-12 17:07:18 -07:00
Gengliang Wang e93b8f02cd [SPARK-32539][INFRA] Disallow FileSystem.get(Configuration conf) in style check by default
### 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>
2020-08-06 05:56:59 +00:00
Warren Zhu 998086c9a1 [SPARK-30794][CORE] Stage Level scheduling: Add ability to set off heap memory
### 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>
2020-07-27 08:16:13 -05:00
Dongjoon Hyun 13c64c2980 [SPARK-32448][K8S][TESTS] Use single version for exec-maven-plugin/scalatest-maven-plugin
### What changes were proposed in this pull request?

Two different versions are used for the same artifacts, `exec-maven-plugin` and `scalatest-maven-plugin`. This PR aims to use the same versions for `exec-maven-plugin` and `scalatest-maven-plugin`. In addition, this PR removes `scala-maven-plugin.version` from `K8s` integration suite because it's unused.

### Why are the changes needed?

This will prevent the mistake which upgrades only one place and forgets the others.

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

No.

### How was this patch tested?

Pass the Jenkins K8S IT.

Closes #29248 from dongjoon-hyun/SPARK-32448.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-26 19:25:41 -07:00
Sean Owen be2eca22e9 [SPARK-32398][TESTS][CORE][STREAMING][SQL][ML] Update to scalatest 3.2.0 for Scala 2.13.3+
### What changes were proposed in this pull request?

Updates to scalatest 3.2.0. Though it looks large, it is 99% changes to the new location of scalatest classes.

### Why are the changes needed?

3.2.0+ has a fix that is required for Scala 2.13.3+ compatibility.

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

No, only affects tests.

### How was this patch tested?

Existing tests.

Closes #29196 from srowen/SPARK-32398.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-23 16:20:17 -07:00
Stijn De Haes 0432379f99 [SPARK-24266][K8S] Restart the watcher when we receive a version changed from k8s
### What changes were proposed in this pull request?

Restart the watcher when it failed with a HTTP_GONE code from the kubernetes api. Which means a resource version has changed.

For more relevant information see here: https://github.com/fabric8io/kubernetes-client/issues/1075

### Why are the changes needed?

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

No

### How was this patch tested?

Running spark-submit to a k8s cluster.

Not sure how to make an automated test for this. If someone can help me out that would be great.

Closes #28423 from stijndehaes/bugfix/k8s-submit-resource-version-change.

Authored-by: Stijn De Haes <stijndehaes@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-07-21 16:34:30 -07:00
maruilei ffdca8285e [SPARK-32367][K8S][TESTS] Correct the spelling of parameter in KubernetesTestComponents
### What changes were proposed in this pull request?

Correct the spelling of parameter 'spark.executor.instances' in KubernetesTestComponents

### Why are the changes needed?

Parameter spelling error

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

No.

### How was this patch tested?

Test is not needed.

Closes #29164 from merrily01/SPARK-32367.

Authored-by: maruilei <maruilei@jd.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-20 13:48:57 -07:00
Holden Karau a4ca355af8 [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown
### What is changed?

This pull request adds the ability to migrate shuffle files during Spark's decommissioning. The design document associated with this change is at https://docs.google.com/document/d/1xVO1b6KAwdUhjEJBolVPl9C6sLj7oOveErwDSYdT-pE .

To allow this change the `MapOutputTracker` has been extended to allow the location of shuffle files to be updated with `updateMapOutput`. When a shuffle block is put, a block update message will be sent which triggers the `updateMapOutput`.

Instead of rejecting remote puts of shuffle blocks `BlockManager` delegates the storage of shuffle blocks to it's shufflemanager's resolver (if supported). A new, experimental, trait is added for shuffle resolvers to indicate they handle remote putting of blocks.

The existing block migration code is moved out into a separate file, and a producer/consumer model is introduced for migrating shuffle files from the host as quickly as possible while not overwhelming other executors.

### Why are the changes needed?

Recomputting shuffle blocks can be expensive, we should take advantage of our decommissioning time to migrate these blocks.

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

This PR introduces two new configs parameters, `spark.storage.decommission.shuffleBlocks.enabled` & `spark.storage.decommission.rddBlocks.enabled` that control which blocks should be migrated during storage decommissioning.

### How was this patch tested?

New unit test & expansion of the Spark on K8s decom test to assert that decommisioning with shuffle block migration means that the results are not recomputed even when the original executor is terminated.

This PR is a cleaned-up version of the previous WIP PR I made https://github.com/apache/spark/pull/28331 (thanks to attilapiros for his very helpful reviewing on it :)).

Closes #28708 from holdenk/SPARK-20629-copy-shuffle-data-when-nodes-are-being-shutdown-cleaned-up.

Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Co-authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Co-authored-by: Attila Zsolt Piros <attilazsoltpiros@apiros-mbp16.lan>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-07-19 21:33:13 -07:00
Sean Owen ee624821a9 [SPARK-29292][YARN][K8S][MESOS] Fix Scala 2.13 compilation for remaining modules
### What changes were proposed in this pull request?

See again the related PRs like https://github.com/apache/spark/pull/28971
This completes fixing compilation for 2.13 for all but `repl`, which is a separate task.

### Why are the changes needed?

Eventually, we need to support a Scala 2.13 build, perhaps in Spark 3.1.

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

No

### How was this patch tested?

Existing tests. (2.13 was not tested; this is about getting it to compile without breaking 2.12)

Closes #29147 from srowen/SPARK-29292.4.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-18 15:08:00 -07:00
Dongjoon Hyun fb51925123 [SPARK-32335][K8S][TESTS] Remove Python2 test from K8s IT
### What changes were proposed in this pull request?

This PR aims to remove Python 2 test case from K8s IT.

### Why are the changes needed?

Since Apache Spark 3.1.0 dropped Python 2.7, 3.4 and 3.5 support officially via SPARK-32138, K8s IT fails.

```
KubernetesSuite:
- Run SparkPi with no resources
- Run SparkPi with a very long application name.
- Use SparkLauncher.NO_RESOURCE
- Run SparkPi with a master URL without a scheme.
- Run SparkPi with an argument.
- Run SparkPi with custom labels, annotations, and environment variables.
- All pods have the same service account by default
- Run extraJVMOptions check on driver
- Run SparkRemoteFileTest using a remote data file
- Run SparkPi with env and mount secrets.
- Run PySpark on simple pi.py example
- Run PySpark with Python2 to test a pyfiles example *** FAILED ***
  The code passed to eventually never returned normally. Attempted 113 times over 2.0014854648999996 minutes. Last failure message: false was not true. (KubernetesSuite.scala:370)
- Run PySpark with Python3 to test a pyfiles example
- Run PySpark with memory customization
- Run in client mode.
- Start pod creation from template
- PVs with local storage
- Launcher client dependencies
- Test basic decommissioning
- Run SparkR on simple dataframe.R example
Run completed in 11 minutes, 15 seconds.
Total number of tests run: 20
Suites: completed 2, aborted 0
Tests: succeeded 19, failed 1, canceled 0, ignored 0, pending 0
*** 1 TEST FAILED ***
```

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

No.

### How was this patch tested?

Pass Jenkins K8s IT.

Closes #29136 from dongjoon-hyun/SPARK-32335.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-16 11:21:14 -07:00
Erik Krogen cf22d947fb [SPARK-32036] Replace references to blacklist/whitelist language with more appropriate terminology, excluding the blacklisting feature
### What changes were proposed in this pull request?

This PR will remove references to these "blacklist" and "whitelist" terms besides the blacklisting feature as a whole, which can be handled in a separate JIRA/PR.

This touches quite a few files, but the changes are straightforward (variable/method/etc. name changes) and most quite self-contained.

### Why are the changes needed?

As per discussion on the Spark dev list, it will be beneficial to remove references to problematic language that can alienate potential community members. One such reference is "blacklist" and "whitelist". While it seems to me that there is some valid debate as to whether these terms have racist origins, the cultural connotations are inescapable in today's world.

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

In the test file `HiveQueryFileTest`, a developer has the ability to specify the system property `spark.hive.whitelist` to specify a list of Hive query files that should be tested. This system property has been renamed to `spark.hive.includelist`. The old property has been kept for compatibility, but will log a warning if used. I am open to feedback from others on whether keeping a deprecated property here is unnecessary given that this is just for developers running tests.

### How was this patch tested?

Existing tests should be suitable since no behavior changes are expected as a result of this PR.

Closes #28874 from xkrogen/xkrogen-SPARK-32036-rename-blacklists.

Authored-by: Erik Krogen <ekrogen@linkedin.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-15 11:40:55 -05:00
HyukjinKwon 4ad9bfd53b [SPARK-32138] Drop Python 2.7, 3.4 and 3.5
### What changes were proposed in this pull request?

This PR aims to drop Python 2.7, 3.4 and 3.5.

Roughly speaking, it removes all the widely known Python 2 compatibility workarounds such as `sys.version` comparison, `__future__`. Also, it removes the Python 2 dedicated codes such as `ArrayConstructor` in Spark.

### Why are the changes needed?

 1. Unsupport EOL Python versions
 2. Reduce maintenance overhead and remove a bit of legacy codes and hacks for Python 2.
 3. PyPy2 has a critical bug that causes a flaky test, SPARK-28358 given my testing and investigation.
 4. Users can use Python type hints with Pandas UDFs without thinking about Python version
 5. Users can leverage one latest cloudpickle, https://github.com/apache/spark/pull/28950. With Python 3.8+ it can also leverage C pickle.

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

Yes, users cannot use Python 2.7, 3.4 and 3.5 in the upcoming Spark version.

### How was this patch tested?

Manually tested and also tested in Jenkins.

Closes #28957 from HyukjinKwon/SPARK-32138.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-14 11:22:44 +09:00
Holden Karau 90ac9f975b [SPARK-32004][ALL] Drop references to slave
### What changes were proposed in this pull request?

This change replaces the world slave with alternatives matching the context.

### Why are the changes needed?

There is no need to call things slave, we might as well use better clearer names.

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

Yes, the ouput JSON does change. To allow backwards compatibility this is an additive change.
The shell scripts for starting & stopping workers are renamed, and for backwards compatibility old scripts are added to call through to the new ones while printing a deprecation message to stderr.

### How was this patch tested?

Existing tests.

Closes #28864 from holdenk/SPARK-32004-drop-references-to-slave.

Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-07-13 14:05:33 -07:00
Pavithraramachandran d7d5bdfd79 [SPARK-32103][CORE] Support IPv6 host/port in core module
### What changes were proposed in this pull request?
In IPv6 scenario, the current logic to split hostname and port is not correct.

### Why are the changes needed?
to support IPV6 deployment scenario

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

### How was this patch tested?
UT and IPV6 spark deployment with yarn

Closes #28931 from PavithraRamachandran/ipv6_issue.

Authored-by: Pavithraramachandran <pavi.rams@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-10 13:55:20 -07:00
Rajat Ahuja ced8e0e662 [SPARK-29465][YARN][WEBUI] Adding Check to not to set UI port (spark.ui.port) property if mentioned explicitly
## What changes were proposed in this pull request?
When a Spark Job launched in Cluster mode with Yarn, Application Master sets spark.ui.port port to 0 which means Driver's web UI gets any random port even if we want to explicitly set the Port range for Driver's Web UI

## Why are the changes needed?
We access Spark Web UI via Knox Proxy, and there are firewall restrictions due to which we can not access Spark Web UI since Web UI port range gets random port even if we set explicitly.

This Change will check if there is a specified port range explicitly mentioned so that it does not assign a random port.

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

## How was this patch tested?
Local Tested.

Closes #28880 from rajatahujaatinmobi/ahujarajat261/SPARK-32039-change-yarn-webui-port-range-with-property-latest-spark.

Authored-by: Rajat Ahuja <rahuja@twitter.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-01 18:28:14 -07:00