Commit graph

160 commits

Author SHA1 Message Date
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
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
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
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
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
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 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
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 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
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
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 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
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
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
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
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
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
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
Udbhav30 d2a656c81e [SPARK-27702][K8S] Allow using some alternatives for service accounts
## What changes were proposed in this pull request?
To allow alternatives to serviceaccounts

### Why are the changes needed?
Although we provide some authentication configuration, such as spark.kubernetes.authenticate.driver.mounted.oauthTokenFile, spark.kubernetes.authenticate.driver.mounted.caCertFile, etc.
But there is a bug as we forced the service account so when we use one of them, driver still use the KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH file, and the error look like bellow:

the KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH serviceAccount not exists

### Does this PR introduce any user-facing change?
Yes user can now use `spark.kubernetes.authenticate.driver.mounted.caCertFile`
or token file by `spark.kubernetes.authenticate.driver.mounted.oauthTokenFile`

## How was this patch tested?
Manually passed the certificates using `spark.kubernetes.authenticate.driver.mounted.caCertFile`
or token file by `spark.kubernetes.authenticate.driver.mounted.oauthTokenFile` if there is no default service account available.

Closes #24601 from Udbhav30/serviceaccount.

Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-20 19:20:54 -07:00
Dongjoon Hyun 64ffc66496
[SPARK-31786][K8S][BUILD] Upgrade kubernetes-client to 4.9.2
### What changes were proposed in this pull request?

This PR aims to upgrade `kubernetes-client` library to bring the JDK8 related fixes. Please note that JDK11 works fine without any problem.
- https://github.com/fabric8io/kubernetes-client/releases/tag/v4.9.2
  - JDK8 always uses http/1.1 protocol (Prevent OkHttp from wrongly enabling http/2)

### Why are the changes needed?

OkHttp "wrongly" detects the Platform as Jdk9Platform on JDK 8u251.
- https://github.com/fabric8io/kubernetes-client/issues/2212
- https://stackoverflow.com/questions/61565751/why-am-i-not-able-to-run-sparkpi-example-on-a-kubernetes-k8s-cluster

Although there is a workaround `export HTTP2_DISABLE=true` and `Downgrade JDK or K8s`, we had better avoid this problematic situation.

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

No. This will recover the failures on JDK 8u252.

### How was this patch tested?

- [x] Pass the Jenkins UT (https://github.com/apache/spark/pull/28601#issuecomment-632474270)
- [x] Pass the Jenkins K8S IT with the K8s 1.13 (https://github.com/apache/spark/pull/28601#issuecomment-632438452)
- [x] Manual testing with K8s 1.17.3. (Below)

**v1.17.6 result (on Minikube)**
```
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
- 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 completed in 8 minutes, 27 seconds.
Total number of tests run: 19
Suites: completed 2, aborted 0
Tests: succeeded 19, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #28601 from dongjoon-hyun/SPARK-K8S-CLIENT.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-23 11:07:45 -07:00
Thomas Graves b64688ebba [SPARK-29303][WEB UI] Add UI support for stage level scheduling
### What changes were proposed in this pull request?

This adds UI updates to support stage level scheduling and ResourceProfiles. 3 main things have been added. ResourceProfile id added to the Stage page, the Executors page now has an optional selectable column to show the ResourceProfile Id of each executor, and the Environment page now has a section with the ResourceProfile ids.  Along with this the rest api for environment page was updated to include the Resource profile information.

I debating on splitting the resource profile information into its own page but I wasn't sure it called for a completely separate page. Open to peoples thoughts on this.

Screen shots:
![Screen Shot 2020-04-01 at 3 07 46 PM](https://user-images.githubusercontent.com/4563792/78185169-469a7000-7430-11ea-8b0c-d9ede2d41df8.png)
![Screen Shot 2020-04-01 at 3 08 14 PM](https://user-images.githubusercontent.com/4563792/78185175-48fcca00-7430-11ea-8d1d-6b9333700f32.png)
![Screen Shot 2020-04-01 at 3 09 03 PM](https://user-images.githubusercontent.com/4563792/78185176-4a2df700-7430-11ea-92d9-73c382bb0f32.png)
![Screen Shot 2020-04-01 at 11 05 48 AM](https://user-images.githubusercontent.com/4563792/78185186-4dc17e00-7430-11ea-8962-f749dd47ea60.png)

### Why are the changes needed?

For user to be able to know what resource profile was used with which stage and executors. The resource profile information is also available so user debugging can see exactly what resources were requested with that profile.

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

Yes, UI updates.

### How was this patch tested?

Unit tests and tested on yarn both active applications and with the history server.

Closes #28094 from tgravescs/SPARK-29303-pr.

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-05-21 13:11:35 -05:00
Dongjoon Hyun c8f3bd861d
[SPARK-31696][K8S] Support driver service annotation in K8S
### What changes were proposed in this pull request?

This PR aims to add `spark.kubernetes.driver.service.annotation` like `spark.kubernetes.driver.service.annotation`.

### Why are the changes needed?

Annotations are used in many ways. One example is that Prometheus monitoring system search metric endpoint via annotation.
- https://github.com/helm/charts/tree/master/stable/prometheus#scraping-pod-metrics-via-annotations

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

Yes. The documentation is added.

### How was this patch tested?

Pass Jenkins with the updated unit tests.

Closes #28518 from dongjoon-hyun/SPARK-31696.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-13 13:59:42 -07:00
Dongjoon Hyun 85dad37f69 [SPARK-31601][K8S] Fix spark.kubernetes.executor.podNamePrefix to work
### What changes were proposed in this pull request?

This PR aims to fix `spark.kubernetes.executor.podNamePrefix` to work.

### Why are the changes needed?

Currently, the configuration is broken like the following.
```
bin/spark-submit \
--master k8s://$K8S_MASTER \
--deploy-mode cluster \
--name spark-pi \
--class org.apache.spark.examples.SparkPi \
-c spark.kubernetes.container.image=spark:pr \
-c spark.kubernetes.driver.pod.name=mypod \
-c spark.kubernetes.executor.podNamePrefix=mypod \
local:///opt/spark/examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar
```

**BEFORE SPARK-31601**
```
pod/mypod                              1/1     Running     0          9s
pod/spark-pi-7469dd71c499fafb-exec-1   1/1     Running     0          4s
pod/spark-pi-7469dd71c499fafb-exec-2   1/1     Running     0          4s
```

**AFTER SPARK-31601**
```
pod/mypod                              1/1     Running     0          8s
pod/mypod-exec-1                       1/1     Running     0          3s
pod/mypod-exec-2                       1/1     Running     0          3s
```

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

Yes. This is a bug fix. The conf will work as described in the documentation.

### How was this patch tested?

Pass the Jenkins and run the above comment manually.

Closes #28401 from dongjoon-hyun/SPARK-31601.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Prashant Sharma <prashsh1@in.ibm.com>
2020-04-30 09:15:12 +05:30
Marcelo Vanzin b8ccd75524
[SPARK-29905][K8S] Improve pod lifecycle manager behavior with dynamic allocation
This issue mainly shows up when you enable dynamic allocation:
because there are many executor state changes (because of executors
being requested and starting to run, and later stopped), the lifecycle
manager class could end up logging information about the same executor
multiple times, since the different events would cause the same
executor update to be present in multiple pod snapshots. On top of that,
it could end up making multiple redundant calls into the API server
for the same pod.

Another issue was when the config was set to not delete executor
pods; with dynamic allocation, that means pods keep accumulating
in the API server, and every time the full sync is done by the
polling source, all executors, even the finished ones that Spark
technically does not care about anymore, would be processed.

The change modifies the lifecycle monitor so that it:

- logs executor updates a single time, even if it shows up in
  multiple snapshots, by checking whether the state change
  happened before.
- marks finished-but-not-deleted-in-k8s executors with a label
  so that they can be easily filtered out.

This reduces the amount of logging done by the lifecycle manager,
which is a minor thing in general since the logs are at debug level.
But it also reduces the amount of data that needs to be fetched
from the API server under certain configurations, and overall
reduces interaction with the API server when dynamic allocation is on.

There's also a change in the snapshot store to ensure that the
same subscriber is not called concurrently. That is kind of a bug,
since it means subscribers could be processing snapshots out of order,
or even that they could block multiple threads (e.g. the allocator
callback was synchronized). I actually ran into the "concurrent calls"
situation in the lifecycle manager during testing, and while it did not
seem to cause problems, it did make for some head scratching while
looking at the logs. It seemed safer to fix that.

Unit tests were updated to check for the changes. Also tested in real
cluster with dynamic allocation on.

Closes #26535 from vanzin/SPARK-29905.

Lead-authored-by: Marcelo Vanzin <vanzin@apache.org>
Co-authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-16 14:15:10 -07:00
Seongjin Cho 7699f765f5
[SPARK-31394][K8S] Adds support for Kubernetes NFS volume mounts
### What changes were proposed in this pull request?
This PR (SPARK-31394) aims to add a new feature that enables mounting of Kubernetes NFS volumes. Most of the codes are just slight modifications from the existing codes for EmptyDir/HostDir/PVC support.

### Why are the changes needed?
Kubernetes supports various kinds of volumes, but Spark for Kubernetes supports only EmptyDir/HostDir/PVC. By adding support for NFS, we can use Spark for Kubernetes with NFS storage.

In order to use NFS with the current Spark using PVC, the user needs to first create an empty new PVC with NFS. Kubernetes' NFS provisioner will create a new empty dir in NFS under some pre-configured dir for this PVC, for example, `/nfs/k8s/sjcho-my-notebook-pvc-dce84888-7a9d-11e6-b1ee-5254001e0c1b`. Then the user should add files to process in the newly created PVC using some file-copying job, and then run the desired Spark job using that populated PVC. And then to get the final results out, the user should run another file-copying job.

This in theory works, but for data analysis tasks, is quite cumbersome. With this change, one could simply use existing files in NFS, say `/nfs/home/sjcho/myfiles10.sstable` from the Spark job directly, and also write the results directly to some existing dir under NFS such as `/nfs/home/sjcho/output`.

This PR doesn't use any features other than the features already provided by Kubernetes itself, so there should be no compatibility issues (other than limited by k8s) between the wide variety of NFS choices. This PR merely enables an existing volume type `nfs` supported officially by Kubernetes, just like Spark is currently supporting `hostPath` and `persistentVolumeClaim` right now.

### Does this PR introduce any user-facing change?
Users can now mount NFS volumes by running commands like:
```
spark-submit \
--conf spark.kubernetes.driver.volumes.nfs.myshare.mount.path=/myshare \
--conf spark.kubernetes.driver.volumes.nfs.myshare.mount.readOnly=false \
--conf spark.kubernetes.driver.volumes.nfs.myshare.options.server=nfs.example.com \
--conf spark.kubernetes.driver.volumes.nfs.myshare.options.path=/storage/myshare \
...
```

### How was this patch tested?
Test cases were added just like the existing EmptyDir support.

The code were tested using minikube using the following script:
https://gist.github.com/w4-sjcho/4ba48f8c35a9685f5307fbd46b2c0656#file-run-test-sh

The script creates a new minikube cluster, launches an NFS server inside the cluster, copy `README.md` file to the NFS share, and run `JavaWordCount` example against the file located in NFS.

Closes #27364 from w4-sjcho/master.

Authored-by: Seongjin Cho <sjcho@wisefour.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-15 03:45:39 -07:00
Prashant Sharma 3799d2b9d8
[SPARK-30715][K8S][TESTS][FOLLOWUP] Update k8s client version in IT as well
### What changes were proposed in this pull request?
This is a follow up for SPARK-30715 . Kubernetes client version in sync in integration-tests and kubernetes/core

### Why are the changes needed?
More than once, the kubernetes client version has gone out of sync between integration tests and kubernetes/core. So brought them up in sync again and added a comment to save us from future need of this additional followup.

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

### How was this patch tested?
Manually.

Closes #27948 from ScrapCodes/follow-up-spark-30715.

Authored-by: Prashant Sharma <prashsh1@in.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-21 18:26:53 -07:00
Holden Karau 57d27e900f
[SPARK-31125][K8S] Terminating pods have a deletion timestamp but they are not yet dead
### What changes were proposed in this pull request?

Change what we consider a deleted pod to not include "Terminating"

### Why are the changes needed?

If we get a new snapshot while a pod is in the process of being cleaned up we shouldn't delete the executor until it is fully terminated.

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

No

### How was this patch tested?

This should be covered by the decommissioning tests in that they currently are flaky because we sometimes delete the executor instead of allowing it to decommission all the way.

I also ran this in a loop locally ~80 times with the only failures being the PV suite because of unrelated minikube mount issues.

Closes #27905 from holdenk/SPARK-31125-Processing-state-snapshots-incorrect.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-17 12:04:06 -07:00
Pedro Rossi ed06d98044
[SPARK-25355][K8S] Add proxy user to driver if present on spark-submit
### What changes were proposed in this pull request?

This PR adds the proxy user on the spark-submit command to the childArgs, so the proxy user can be retrieved and used in the KubernetesAplication to add the proxy user in the driver container args

### Why are the changes needed?

The proxy user when used on the spark submit doesn't work on the Kubernetes environment since it doesn't add the `--proxy-user` argument on the driver container and when I added it manually to the Pod definition it worked just fine.

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

No.

### How was this patch tested?

Tests were added

Closes #27422 from PedroRossi/SPARK-25355.

Authored-by: Pedro Rossi <pgrr@cin.ufpe.br>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-16 21:53:58 -07:00
beliefer 1254c88034 [SPARK-31118][K8S][DOC] Add version information to the configuration of K8S
### What changes were proposed in this pull request?
Add version information to the configuration of `K8S`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.kubernetes.context | 3.0.0 | SPARK-25887 | c542c247bbfe1214c0bf81076451718a9e8931dc#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.master | 3.0.0 | SPARK-30371 | f14061c6a4729ad419902193aa23575d8f17f597#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.namespace | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.container.image | 2.3.0 | SPARK-22994 | b94debd2b01b87ef1d2a34d48877e38ade0969e6#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.container.image | 2.3.0 | SPARK-22807 | fb3636b482be3d0940345b1528c1d5090bbc25e6#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.container.image | 2.3.0 | SPARK-22807 | fb3636b482be3d0940345b1528c1d5090bbc25e6#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.container.image.pullPolicy | 2.3.0 | SPARK-22807 | fb3636b482be3d0940345b1528c1d5090bbc25e6#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.container.image.pullSecrets | 2.4.0 | SPARK-23668 | cccaaa14ad775fb981e501452ba2cc06ff5c0f0a#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.submission.requestTimeout | 3.0.0 | SPARK-27023 | e9e8bb33ef9ad785473ded168bc85867dad4ee70#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.submission.connectionTimeout | 3.0.0 | SPARK-27023 | e9e8bb33ef9ad785473ded168bc85867dad4ee70#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.requestTimeout | 3.0.0 | SPARK-27023 | e9e8bb33ef9ad785473ded168bc85867dad4ee70#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.connectionTimeout | 3.0.0 | SPARK-27023 | e9e8bb33ef9ad785473ded168bc85867dad4ee70#diff-6e882d5561424e7e6651eb46f10104b8 |  
KUBERNETES_AUTH_DRIVER_CONF_PREFIX.serviceAccountName | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 | spark.kubernetes.authenticate.driver
KUBERNETES_AUTH_EXECUTOR_CONF_PREFIX.serviceAccountName | 3.1.0 | SPARK-30122 | f9f06eee9853ad4b6458ac9d31233e729a1ca226#diff-6e882d5561424e7e6651eb46f10104b8 | spark.kubernetes.authenticate.executor
spark.kubernetes.driver.limit.cores | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.request.cores | 3.0.0 | SPARK-27754 | 1a8c09334db87b0e938c38cd6b59d326bdcab3c3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.submitInDriver | 2.4.0 | SPARK-22839 | f15906da153f139b698e192ec6f82f078f896f1e#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.limit.cores | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.scheduler.name | 3.0.0 | SPARK-29436 | f800fa383131559c4e841bf062c9775d09190935#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.request.cores | 2.4.0 | SPARK-23285 | fe2b7a4568d65a62da6e6eb00fff05f248b4332c#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.pod.name | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.resourceNamePrefix | 3.0.0 | SPARK-25876 | 6be272b75b4ae3149869e19df193675cc4117763#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.podNamePrefix | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.allocation.batch.size | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.allocation.batch.delay | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.lostCheck.maxAttempts | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.submission.waitAppCompletion | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.report.interval | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.apiPollingInterval | 2.4.0 | SPARK-24248 | 270a9a3cac25f3e799460320d0fc94ccd7ecfaea#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.eventProcessingInterval | 2.4.0 | SPARK-24248 | 270a9a3cac25f3e799460320d0fc94ccd7ecfaea#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.memoryOverheadFactor | 2.4.0 | SPARK-23984 | 1a644afbac35c204f9ad55f86999319a9ab458c6#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.pyspark.pythonVersion | 2.4.0 | SPARK-23984 | a791c29bd824adadfb2d85594bc8dad4424df936#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.kerberos.krb5.path | 3.0.0 | SPARK-23257 | 6c9c84ffb9c8d98ee2ece7ba4b010856591d383d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.kerberos.krb5.configMapName | 3.0.0 | SPARK-23257 | 6c9c84ffb9c8d98ee2ece7ba4b010856591d383d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.hadoop.configMapName | 3.0.0 | SPARK-23257 | 6c9c84ffb9c8d98ee2ece7ba4b010856591d383d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.kerberos.tokenSecret.name | 3.0.0 | SPARK-23257 | 6c9c84ffb9c8d98ee2ece7ba4b010856591d383d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.kerberos.tokenSecret.itemKey | 3.0.0 | SPARK-23257 | 6c9c84ffb9c8d98ee2ece7ba4b010856591d383d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.resource.type | 2.4.1 | SPARK-25021 | 9031c784847353051bc0978f63ef4146ae9095ff#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.local.dirs.tmpfs | 3.0.0 | SPARK-25262 | da6fa3828bb824b65f50122a8a0a0d4741551257#diff-6e882d5561424e7e6651eb46f10104b8 | It exists in branch-3.0, but in pom.xml it is 2.4.0-snapshot
spark.kubernetes.driver.podTemplateFile | 3.0.0 | SPARK-24434 | f6cc354d83c2c9a757f9b507aadd4dbdc5825cca#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.podTemplateFile | 3.0.0 | SPARK-24434 | f6cc354d83c2c9a757f9b507aadd4dbdc5825cca#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.podTemplateContainerName | 3.0.0 | SPARK-24434 | f6cc354d83c2c9a757f9b507aadd4dbdc5825cca#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.podTemplateContainerName | 3.0.0 | SPARK-24434 | f6cc354d83c2c9a757f9b507aadd4dbdc5825cca#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.deleteOnTermination | 3.0.0 | SPARK-25515 | 0c2935b01def8a5f631851999d9c2d57b63763e6#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.dynamicAllocation.deleteGracePeriod | 3.0.0 | SPARK-28487 | 0343854f54b48b206ca434accec99355011560c2#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.appKillPodDeletionGracePeriod | 3.0.0 | SPARK-24793 | 05168e725d2a17c4164ee5f9aa068801ec2454f4#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.file.upload.path | 3.0.0 | SPARK-23153 | 5e74570c8f5e7dfc1ca1c53c177827c5cea57bf1#diff-6e882d5561424e7e6651eb46f10104b8 |  
The following appears in the document |   |   |   |  
spark.kubernetes.authenticate.submission.caCertFile | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.submission.clientKeyFile | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.submission.clientCertFile | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.submission.oauthToken | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.submission.oauthTokenFile | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.driver.caCertFile | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.driver.clientKeyFile | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.driver.clientCertFile | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.driver.oauthToken | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.driver.oauthTokenFile | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.driver.mounted.caCertFile | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.driver.mounted.clientKeyFile | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.driver.mounted.clientCertFile | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.driver.mounted.oauthTokenFile | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.caCertFile | 2.4.0 | SPARK-23146 | 571a6f0574e50e53cea403624ec3795cd03aa204#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.clientKeyFile | 2.4.0 | SPARK-23146 | 571a6f0574e50e53cea403624ec3795cd03aa204#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.clientCertFile | 2.4.0 | SPARK-23146 | 571a6f0574e50e53cea403624ec3795cd03aa204#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.oauthToken | 2.4.0 | SPARK-23146 | 571a6f0574e50e53cea403624ec3795cd03aa204#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.authenticate.oauthTokenFile | 2.4.0 | SPARK-23146 | 571a6f0574e50e53cea403624ec3795cd03aa204#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.label.[LabelName] | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.annotation.[AnnotationName] | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.label.[LabelName] | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.annotation.[AnnotationName] | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.node.selector.[labelKey] | 2.3.0 | SPARK-18278 | e9b2070ab2d04993b1c0c1d6c6aba249e6664c8d#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driverEnv.[EnvironmentVariableName] | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.secrets.[SecretName] | 2.3.0 | SPARK-22757 | 171f6ddadc6185ffcc6ad82e5f48952fb49095b2#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.secrets.[SecretName] | 2.3.0 | SPARK-22757 | 171f6ddadc6185ffcc6ad82e5f48952fb49095b2#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.secretKeyRef.[EnvName] | 2.4.0 | SPARK-24232 | 21e1fc7d4aed688d7b685be6ce93f76752159c98#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.secretKeyRef.[EnvName] | 2.4.0 | SPARK-24232 | 21e1fc7d4aed688d7b685be6ce93f76752159c98#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.path | 2.4.0 | SPARK-23529 | 5ff1b9ba1983d5601add62aef64a3e87d07050eb#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.subPath | 3.0.0 | SPARK-25960 | 3df307aa515b3564686e75d1b71754bbcaaf2dec#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.readOnly | 2.4.0 | SPARK-23529 | 5ff1b9ba1983d5601add62aef64a3e87d07050eb#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].options.[OptionName] | 2.4.0 | SPARK-23529 | 5ff1b9ba1983d5601add62aef64a3e87d07050eb#diff-b5527f236b253e0d9f5db5164bdb43e9 |  
spark.kubernetes.executor.volumes.[VolumeType].[VolumeName].mount.path | 2.4.0 | SPARK-23529 | 5ff1b9ba1983d5601add62aef64a3e87d07050eb#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.volumes.[VolumeType].[VolumeName].mount.subPath | 3.0.0 | SPARK-25960 | 3df307aa515b3564686e75d1b71754bbcaaf2dec#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.volumes.[VolumeType].[VolumeName].mount.readOnly | 2.4.0 | SPARK-23529 | 5ff1b9ba1983d5601add62aef64a3e87d07050eb#diff-6e882d5561424e7e6651eb46f10104b8 |  
spark.kubernetes.executor.volumes.[VolumeType].[VolumeName].options.[OptionName] | 2.4.0 | SPARK-23529 | 5ff1b9ba1983d5601add62aef64a3e87d07050eb#diff-b5527f236b253e0d9f5db5164bdb43e9 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27875 from beliefer/add-version-to-k8s-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-12 09:54:08 +09:00
gatorsmile 28b8713036 [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT
### What changes were proposed in this pull request?
This patch is to bump the master branch version to 3.1.0-SNAPSHOT.

### Why are the changes needed?
N/A

### Does this PR introduce any user-facing change?
N/A

### How was this patch tested?
N/A

Closes #27698 from gatorsmile/updateVersion.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-25 19:44:31 -08:00
Holden Karau d273a2bb0f [SPARK-20628][CORE][K8S] Start to improve Spark decommissioning & preemption support
This PR is based on an existing/previou PR - https://github.com/apache/spark/pull/19045

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

This changes adds a decommissioning state that we can enter when the cloud provider/scheduler lets us know we aren't going to be removed immediately but instead will be removed soon. This concept fits nicely in K8s and also with spot-instances on AWS / preemptible instances all of which we can get a notice that our host is going away. For now we simply stop scheduling jobs, in the future we could perform some kind of migration of data during scale-down, or at least stop accepting new blocks to cache.

There is a design document at https://docs.google.com/document/d/1xVO1b6KAwdUhjEJBolVPl9C6sLj7oOveErwDSYdT-pE/edit?usp=sharing

### Why are the changes needed?

With more move to preemptible multi-tenancy, serverless environments, and spot-instances better handling of node scale down is required.

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

There is no API change, however an additional configuration flag is added to enable/disable this behaviour.

### How was this patch tested?

New integration tests in the Spark K8s integration testing. Extension of the AppClientSuite to test decommissioning seperate from the K8s.

Closes #26440 from holdenk/SPARK-20628-keep-track-of-nodes-which-are-going-to-be-shutdown-r4.

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-02-14 12:36:52 -08:00
Thomas Graves 496f6ac860 [SPARK-29148][CORE] Add stage level scheduling dynamic allocation and scheduler backend changes
### What changes were proposed in this pull request?

This is another PR for stage level scheduling. In particular this adds changes to the dynamic allocation manager and the scheduler backend to be able to track what executors are needed per ResourceProfile.  Note the api is still private to Spark until the entire feature gets in, so this functionality will be there but only usable by tests for profiles other then the DefaultProfile.

The main changes here are simply tracking things on a ResourceProfile basis as well as sending the executor requests to the scheduler backend for all ResourceProfiles.

I introduce a ResourceProfileManager in this PR that will track all the actual ResourceProfile objects so that we can keep them all in a single place and just pass around and use in datastructures the resource profile id. The resource profile id can be used with the ResourceProfileManager to get the actual ResourceProfile contents.

There are various places in the code that use executor "slots" for things.  The ResourceProfile adds functionality to keep that calculation in it.   This logic is more complex then it should due to standalone mode and mesos coarse grained not setting the executor cores config. They default to all cores on the worker, so calculating slots is harder there.
This PR keeps the functionality to make the cores the limiting resource because the scheduler still uses that for "slots" for a few things.

This PR does also add the resource profile id to the Stage and stage info classes to be able to test things easier.   That full set of changes will come with the scheduler PR that will be after this one.

The PR stops at the scheduler backend pieces for the cluster manager and the real YARN support hasn't been added in this PR, that again will be in a separate PR, so this has a few of the API changes up to the cluster manager and then just uses the default profile requests to continue.

The code for the entire feature is here for reference: https://github.com/apache/spark/pull/27053/files although it needs to be upmerged again as well.

### Why are the changes needed?

Needed for stage level scheduling feature.

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

No user facing api changes added here.

### How was this patch tested?

Lots of unit tests and manually testing. I tested on yarn, k8s, standalone, local modes. Ran both failure and success cases.

Closes #27313 from tgravescs/SPARK-29148.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-02-12 16:45:42 -06:00
yudovin f9f06eee98 [SPARK-30122][K8S] Support spark.kubernetes.authenticate.executor.serviceAccountName
### What changes were proposed in this pull request?

Currently, it doesn't seem to be possible to have Spark Driver set the serviceAccountName for executor pods it launches.

### Why are the changes needed?

it will allow settings serviceAccountName for executors pods.

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

No

### How was this patch tested?

It was covered by unit test.

Closes #27034 from ayudovin/srevice-account-name-for-executor-pods.

Authored-by: yudovin <artsiom.yudovin@profitero.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-05 14:16:59 -08:00
Onur Satici 86fdb818bf [SPARK-30715][K8S] Bump fabric8 to 4.7.1
### What changes were proposed in this pull request?
Bump fabric8 kubernetes-client to 4.7.1

### Why are the changes needed?
New fabric8 version brings support for Kubernetes 1.17 clusters.
Full release notes:
- https://github.com/fabric8io/kubernetes-client/releases/tag/v4.7.0
- https://github.com/fabric8io/kubernetes-client/releases/tag/v4.7.1

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

### How was this patch tested?
Existing unit and integration tests cover creation of K8S objects. Adjusted them to work with the new fabric8 version

Closes #27443 from onursatici/os/bump-fabric8.

Authored-by: Onur Satici <onursatici@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-05 01:17:30 -08:00
Thomas Graves 878094f972 [SPARK-30689][CORE][YARN] Add resource discovery plugin api to support YARN versions with resource scheduling
### What changes were proposed in this pull request?

This change is to allow custom resource scheduler (GPUs,FPGAs,etc) resource discovery to be more flexible. Users are asking for it to work with hadoop 2.x versions that do not support resource scheduling in YARN and/or also they may not run in an isolated environment.
This change creates a plugin api that users can write their own resource discovery class that allows a lot more flexibility. The user can chain plugins for different resource types. The user specified plugins execute in the order specified and will fall back to use the discovery script plugin if they don't return information for a particular resource.

I had to open up a few of the classes to be public and change them to not be case classes and make them developer api in order for the the plugin to get enough information it needs.

I also relaxed the yarn side so that if yarn isn't configured for resource scheduling we just warn and go on. This helps users that have yarn 3.1 but haven't configured the resource scheduling side on their cluster yet, or aren't running in isolated environment.

The user would configured this like:
--conf spark.resources.discovery.plugin="org.apache.spark.resource.ResourceDiscoveryFPGAPlugin, org.apache.spark.resource.ResourceDiscoveryGPUPlugin"

Note the executor side had to be wrapped with a classloader to make sure we include the user classpath for jars they specified on submission.

Note this is more flexible because the discovery script has limitations such as spawning it in a separate process. This means if you are trying to allocate resources in that process they might be released when the script returns. Other things are the class makes it more flexible to be able to integrate with existing systems and solutions for assigning resources.

### Why are the changes needed?

to more easily use spark resource scheduling with older versions of hadoop or in non-isolated enivronments.

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

Yes a plugin api

### How was this patch tested?

Unit tests added and manual testing done on yarn and standalone modes.

Closes #27410 from tgravescs/hadoop27spark3.

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-01-31 22:20:28 -06:00
Jiaxin Shan f86a1b9590 [SPARK-30626][K8S] Add SPARK_APPLICATION_ID into driver pod env
### What changes were proposed in this pull request?
Add SPARK_APPLICATION_ID environment when spark configure driver pod.

### Why are the changes needed?
Currently, driver doesn't have this in environments and it's no convenient to retrieve spark id.
The use case is we want to look up spark application id and create application folder and redirect driver logs to application folder.

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

### How was this patch tested?
unit tested. I also build new distribution and container image to kick off a job in Kubernetes and I do see SPARK_APPLICATION_ID added there. .

Closes #27347 from Jeffwan/SPARK-30626.

Authored-by: Jiaxin Shan <seedjeffwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-24 12:00:30 -08:00
xushiwei 00425595 f14061c6a4 [SPARK-30371][K8S] Add spark.kubernetes.driver.master conf
### What changes were proposed in this pull request?

make KUBERNETES_MASTER_INTERNAL_URL configurable

### Why are the changes needed?

we do not always use the default port number 443 to access our kube-apiserver, and even in some mulit-tenant cluster,  people do not use the service `kubernetes.default.svc` to access the kube-apiserver, so make the internal master configurable is necessary。

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

user can configure the internal master url by
```
--conf spark.kubernetes.internal.master=https://kubernetes.default.svc:6443
```

### How was this patch tested?

run in multi-cluster that do not use the https://kubernetes.default.svc to access the kube-apiserver

Closes #27029 from wackxu/internalmaster.

Authored-by: xushiwei 00425595 <xushiwei5@huawei.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-19 14:14:45 -08:00