Commit graph

38 commits

Author SHA1 Message Date
gatorsmile bb2f069cf2 [SPARK-25436] Bump master branch version to 2.5.0-SNAPSHOT
## What changes were proposed in this pull request?
In the dev list, we can still discuss whether the next version is 2.5.0 or 3.0.0. Let us first bump the master branch version to `2.5.0-SNAPSHOT`.

## How was this patch tested?
N/A

Closes #22426 from gatorsmile/bumpVersionMaster.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-15 16:24:02 -07:00
Stavros Kontopoulos 3e75a9fa24 [SPARK-25295][K8S] Fix executor names collision
## What changes were proposed in this pull request?
Fixes the collision issue with spark executor names in client mode, see SPARK-25295 for the details.
It follows the cluster name convention as app-name will be used as the prefix and if that is not defined we use "spark" as the default prefix. Eg. `spark-pi-1536781360723-exec-1` where spark-pi is the name of the app passed at the config side or transformed if it contains illegal characters.

Also fixes the issue with spark app name having spaces in cluster mode.
If you run the Spark Pi test in client mode it passes.
The tricky part is the user may set the app name:
3030b82c89/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala (L30)
If i do:

```
./bin/spark-submit
...
 --deploy-mode cluster --name "spark pi"
...
```
it will fail as the app name is used for the prefix of driver's pod name and it cannot have spaces (according to k8s conventions).

## How was this patch tested?

Manually by running spark job in client mode.
To reproduce do:
```
kubectl create -f service.yaml
kubectl create -f pod.yaml
```
 service.yaml :
```
kind: Service
apiVersion: v1
metadata:
  name: spark-test-app-1-svc
spec:
  clusterIP: None
  selector:
    spark-app-selector: spark-test-app-1
  ports:
  - protocol: TCP
    name: driver-port
    port: 7077
    targetPort: 7077
  - protocol: TCP
    name: block-manager
    port: 10000
    targetPort: 10000
```
pod.yaml:

```
apiVersion: v1
kind: Pod
metadata:
  name: spark-test-app-1
  labels:
    spark-app-selector: spark-test-app-1
spec:
  containers:
  - name: spark-test
    image: skonto/spark:k8s-client-fix
    imagePullPolicy: Always
    command:
      - 'sh'
      - '-c'
      -  "/opt/spark/bin/spark-submit
              --verbose
              --master k8s://https://kubernetes.default.svc
              --deploy-mode client
              --class org.apache.spark.examples.SparkPi
              --conf spark.app.name=spark
              --conf spark.executor.instances=1
              --conf spark.kubernetes.container.image=skonto/spark:k8s-client-fix
              --conf spark.kubernetes.container.image.pullPolicy=Always
              --conf spark.kubernetes.authenticate.oauthTokenFile=/var/run/secrets/kubernetes.io/serviceaccount/token
              --conf spark.kubernetes.authenticate.caCertFile=/var/run/secrets/kubernetes.io/serviceaccount/ca.crt
              --conf spark.executor.memory=500m
              --conf spark.executor.cores=1
              --conf spark.executor.instances=1
              --conf spark.driver.host=spark-test-app-1-svc.default.svc
              --conf spark.driver.port=7077
              --conf spark.driver.blockManager.port=10000
              local:///opt/spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar 1000000"
```

Closes #22405 from skonto/fix-k8s-client-mode-executor-names.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Yinan Li <ynli@google.com>
2018-09-12 22:02:59 -07:00
Ilan Filonenko 1cfda44825 [SPARK-25021][K8S] Add spark.executor.pyspark.memory limit for K8S
## What changes were proposed in this pull request?

Add spark.executor.pyspark.memory limit for K8S

## How was this patch tested?

Unit and Integration tests

Closes #22298 from ifilonenko/SPARK-25021.

Authored-by: Ilan Filonenko <if56@cornell.edu>
Signed-off-by: Holden Karau <holden@pigscanfly.ca>
2018-09-08 22:18:06 -07:00
Rob Vesse da6fa3828b [SPARK-25262][K8S] Allow SPARK_LOCAL_DIRS to be tmpfs backed on K8S
## What changes were proposed in this pull request?

The default behaviour of Spark on K8S currently is to create `emptyDir` volumes to back `SPARK_LOCAL_DIRS`.  In some environments e.g. diskless compute nodes this may actually hurt performance because these are backed by the Kubelet's node storage which on a diskless node will typically be some remote network storage.

Even if this is enterprise grade storage connected via a high speed interconnect the way Spark uses these directories as scratch space (lots of relatively small short lived files) has been observed to cause serious performance degradation.  Therefore we would like to provide the option to use K8S's ability to instead back these `emptyDir` volumes with `tmpfs`. Therefore this PR adds a configuration option that enables `SPARK_LOCAL_DIRS` to be backed by Memory backed `emptyDir` volumes rather than the default.

Documentation is added to describe both the default behaviour plus this new option and its implications.  One of which is that scratch space then counts towards your pods memory limits and therefore users will need to adjust their memory requests accordingly.

*NB* - This is an alternative version of PR #22256 reduced to just the `tmpfs` piece

## How was this patch tested?

Ran with this option in our diskless compute environments to verify functionality

Author: Rob Vesse <rvesse@dotnetrdf.org>

Closes #22323 from rvesse/SPARK-25262-tmpfs.
2018-09-06 16:18:59 -07:00
Rob Vesse 27d3b0a51c [SPARK-25222][K8S] Improve container status logging
## What changes were proposed in this pull request?

Currently when running Spark on Kubernetes a logger is run by the client that watches the K8S API for events related to the Driver pod and logs them.  However for the container status aspect of the logging this simply dumps the raw object which is not human readable e.g.

![screen shot 2018-08-24 at 10 37 46](https://user-images.githubusercontent.com/2104864/44577799-e0486880-a789-11e8-9ae9-fdeddacbbea8.png)
![screen shot 2018-08-24 at 10 38 14](https://user-images.githubusercontent.com/2104864/44577800-e0e0ff00-a789-11e8-81f5-3bb315dbbdb1.png)

This is despite the fact that the logging class in question actually has methods to pretty print this information but only invokes these at the end of a job.

This PR improves the logging to always use the pretty printing methods, additionally modifying them to include further useful information provided by the K8S API.

A similar issue also exists when tasks are lost that will be addressed by further commits to this PR

- [x] Improved `LoggingPodStatusWatcher`
- [x] Improved container status on task failure

## How was this patch tested?

Built and launched jobs with the updated Spark client and observed the new human readable output:

![screen shot 2018-08-24 at 11 09 32](https://user-images.githubusercontent.com/2104864/44579429-5353de00-a78e-11e8-9228-c750af8e6311.png)
![screen shot 2018-08-24 at 11 09 42](https://user-images.githubusercontent.com/2104864/44579430-5353de00-a78e-11e8-8fce-d5bb2a3ae65f.png)
![screen shot 2018-08-24 at 11 10 13](https://user-images.githubusercontent.com/2104864/44579431-53ec7480-a78e-11e8-9fa2-aeabc5b28ec4.png)
![screen shot 2018-08-24 at 17 47 44](https://user-images.githubusercontent.com/2104864/44596922-db090f00-a7c5-11e8-910c-bc2339f5a196.png)

Suggested reviewers: liyinan926 mccheah

Author: Rob Vesse <rvesse@dotnetrdf.org>

Closes #22215 from rvesse/SPARK-25222.
2018-09-06 16:15:11 -07:00
Ilan Filonenko e1d72f2c07 [SPARK-25264][K8S] Fix comma-delineated arguments passed into PythonRunner and RRunner
## What changes were proposed in this pull request?

Fixes the issue brought up in https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/issues/273 where the arguments were being comma-delineated, which was incorrect wrt to the PythonRunner and RRunner.

## How was this patch tested?

Modified unit test to test this change.

Author: Ilan Filonenko <if56@cornell.edu>

Closes #22257 from ifilonenko/SPARK-25264.
2018-08-31 15:46:45 -07:00
Ilan Filonenko ba84bcb2c4 [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s
## What changes were proposed in this pull request?

Introducing R Bindings for Spark R on K8s

- [x] Running SparkR Job

## How was this patch tested?

This patch was tested with

- [x] Unit Tests
- [x] Integration Tests

## Example:

Commands to run example spark job:
1. `dev/make-distribution.sh --pip --r --tgz -Psparkr -Phadoop-2.7 -Pkubernetes`
2. `bin/docker-image-tool.sh -m -t testing build`
3.
```
bin/spark-submit \
    --master k8s://https://192.168.64.33:8443 \
    --deploy-mode cluster \
    --name spark-r \
    --conf spark.executor.instances=1 \
    --conf spark.kubernetes.container.image=spark-r:testing \
    local:///opt/spark/examples/src/main/r/dataframe.R
```

This above spark-submit command works given the distribution. (Will include this integration test in PR once PRB is ready).

Author: Ilan Filonenko <if56@cornell.edu>

Closes #21584 from ifilonenko/spark-r.
2018-08-17 16:04:02 -07:00
Ilan Filonenko a791c29bd8 [SPARK-23984][K8S] Changed Python Version config to be camelCase
## What changes were proposed in this pull request?

Small formatting change to have Python Version be camelCase as per request during PR review.

## How was this patch tested?

Tested with unit and integration tests

Author: Ilan Filonenko <if56@cornell.edu>

Closes #22095 from ifilonenko/spark-py-edits.
2018-08-15 17:52:12 -07:00
Adelbert Chang f5113ea8d7 [SPARK-24960][K8S] explicitly expose ports on driver container
https://issues.apache.org/jira/browse/SPARK-24960
## What changes were proposed in this pull request?

Expose ports explicitly in the driver container. The driver Service created expects to reach the driver Pod at specific ports which before this change, were not explicitly exposed and would likely cause connection issues (see https://github.com/apache-spark-on-k8s/spark/issues/617).

This is a port of the original PR created in the now-deprecated Kubernetes fork: https://github.com/apache-spark-on-k8s/spark/pull/618

## How was this patch tested?

Failure in https://github.com/apache-spark-on-k8s/spark/issues/617 reproduced on Kubernetes 1.6.x and 1.8.x. Built the driver image with this patch and observed fixed https://github.com/apache-spark-on-k8s/spark/issues/617 on Kubernetes 1.6.x.

Author: Adelbert Chang <Adelbert.Chang@target.com>

Closes #21884 from adelbertc/k8s-expose-driver-ports.
2018-08-01 13:57:33 -07:00
mcheah 571a6f0574 [SPARK-23146][K8S] Support client mode.
## What changes were proposed in this pull request?

Support client mode for the Kubernetes scheduler.

Client mode works more or less identically to cluster mode. However, in client mode, the Spark Context needs to be manually bootstrapped with certain properties which would have otherwise been set up by spark-submit in cluster mode. Specifically:

- If the user doesn't provide a driver pod name, we don't add an owner reference. This is for usage when the driver is not running in a pod in the cluster. In such a case, the driver can only provide a best effort to clean up the executors when the driver exits, but cleaning up the resources is not guaranteed. The executor JVMs should exit if the driver JVM exits, but the pods will still remain in the cluster in a COMPLETED or FAILED state.
- The user must provide a host (spark.driver.host) and port (spark.driver.port) that the executors can connect to. When using spark-submit in cluster mode, spark-submit generates the headless service automatically; in client mode, the user is responsible for setting up their own connectivity.

We also change the authentication configuration prefixes for client mode.

## How was this patch tested?

Adding an integration test to exercise client mode support.

Author: mcheah <mcheah@palantir.com>

Closes #21748 from mccheah/k8s-client-mode.
2018-07-25 11:08:41 -07:00
Kazuaki Ishizaki 5ad4735bda [SPARK-24529][BUILD][TEST-MAVEN] Add spotbugs into maven build process
## What changes were proposed in this pull request?

This PR enables a Java bytecode check tool [spotbugs](https://spotbugs.github.io/) to avoid possible integer overflow at multiplication. When an violation is detected, the build process is stopped.
Due to the tool limitation, some other checks will be enabled. In this PR, [these patterns](http://spotbugs-in-kengo-toda.readthedocs.io/en/lqc-list-detectors/detectors.html#findpuzzlers) in `FindPuzzlers` can be detected.

This check is enabled at `compile` phase. Thus, `mvn compile` or `mvn package` launches this check.

## How was this patch tested?

Existing UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21542 from kiszk/SPARK-24529.
2018-07-12 09:52:23 +08:00
Andrew Korzhuev 5ff1b9ba19 [SPARK-23529][K8S] Support mounting volumes
This PR continues #21095 and intersects with #21238. I've added volume mounts as a separate step and added PersistantVolumeClaim support.

There is a fundamental problem with how we pass the options through spark conf to fabric8. For each volume type and all possible volume options we would have to implement some custom code to map config values to fabric8 calls. This will result in big body of code we would have to support and means that Spark will always be somehow out of sync with k8s.

I think there needs to be a discussion on how to proceed correctly (eg use PodPreset instead)

----

Due to the complications of provisioning and managing actual resources this PR addresses only volume mounting of already present resources.

----
- [x] emptyDir support
- [x] Testing
- [x] Documentation
- [x] KubernetesVolumeUtils tests

Author: Andrew Korzhuev <andrew.korzhuev@klarna.com>
Author: madanadit <adit@alluxio.com>

Closes #21260 from andrusha/k8s-vol.
2018-07-10 22:53:44 -07:00
Stavros Kontopoulos 85fe1297e3 [SPARK-24428][K8S] Fix unused code
## What changes were proposed in this pull request?

Remove code that is misleading and is a leftover from a previous implementation.

## How was this patch tested?
Manually.

Author: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>

Closes #21462 from skonto/fix-k8s-docs.
2018-07-02 13:08:16 -07:00
mcheah 42815548c7 [SPARK-24683][K8S] Fix k8s no resource
## What changes were proposed in this pull request?

Make SparkSubmit pass in the main class even if `SparkLauncher.NO_RESOURCE` is the primary resource.

## How was this patch tested?

New integration test written to capture this case.

Author: mcheah <mcheah@palantir.com>

Closes #21660 from mccheah/fix-k8s-no-resource.
2018-07-02 10:24:04 -07:00
mcheah 270a9a3cac [SPARK-24248][K8S] Use level triggering and state reconciliation in scheduling and lifecycle
## What changes were proposed in this pull request?

Previously, the scheduler backend was maintaining state in many places, not only for reading state but also writing to it. For example, state had to be managed in both the watch and in the executor allocator runnable. Furthermore, one had to keep track of multiple hash tables.

We can do better here by:

1. Consolidating the places where we manage state. Here, we take inspiration from traditional Kubernetes controllers. These controllers tend to follow a level-triggered mechanism. This means that the controller will continuously monitor the API server via watches and polling, and on periodic passes, the controller will reconcile the current state of the cluster with the desired state. We implement this by introducing the concept of a pod snapshot, which is a given state of the executors in the Kubernetes cluster. We operate periodically on snapshots. To prevent overloading the API server with polling requests to get the state of the cluster (particularly for executor allocation where we want to be checking frequently to get executors to launch without unbearably bad latency), we use watches to populate snapshots by applying observed events to a previous snapshot to get a new snapshot. Whenever we do poll the cluster, the polled state replaces any existing snapshot - this ensures eventual consistency and mirroring of the cluster, as is desired in a level triggered architecture.

2. Storing less specialized in-memory state in general. Previously we were creating hash tables to represent the state of executors. Instead, it's easier to represent state solely by the snapshots.

## How was this patch tested?

Integration tests should test there's no regressions end to end. Unit tests to be updated, in particular focusing on different orderings of events, particularly accounting for when events come in unexpected ordering.

Author: mcheah <mcheah@palantir.com>

Closes #21366 from mccheah/event-queue-driven-scheduling.
2018-06-14 15:56:21 -07:00
Ilan Filonenko 1a644afbac [SPARK-23984][K8S] Initial Python Bindings for PySpark on K8s
## What changes were proposed in this pull request?

Introducing Python Bindings for PySpark.

- [x] Running PySpark Jobs
- [x] Increased Default Memory Overhead value
- [ ] Dependency Management for virtualenv/conda

## How was this patch tested?

This patch was tested with

- [x] Unit Tests
- [x] Integration tests with [this addition](https://github.com/apache-spark-on-k8s/spark-integration/pull/46)
```
KubernetesSuite:
- Run SparkPi with no resources
- Run SparkPi with a very long application name.
- Run SparkPi with a master URL without a scheme.
- Run SparkPi with an argument.
- Run SparkPi with custom labels, annotations, and environment variables.
- Run SparkPi with a test secret mounted into the driver and executor pods
- Run extraJVMOptions check on driver
- Run SparkRemoteFileTest using a remote data file
- 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 completed in 4 minutes, 28 seconds.
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.
```

Author: Ilan Filonenko <if56@cornell.edu>
Author: Ilan Filonenko <ifilondz@gmail.com>

Closes #21092 from ifilonenko/master.
2018-06-08 11:18:34 -07:00
Stavros Kontopoulos 21e1fc7d4a [SPARK-24232][K8S] Add support for secret env vars
## What changes were proposed in this pull request?

* Allows to refer a secret as an env var.
* Introduces new config properties in the form: spark.kubernetes{driver,executor}.secretKeyRef.ENV_NAME=name:key
  ENV_NAME is case sensitive.

* Updates docs.
* Adds required unit tests.

## How was this patch tested?
Manually tested and confirmed that the secrets exist in driver's and executor's container env.
Also job finished successfully.
First created a secret with the following yaml:
```
apiVersion: v1
kind: Secret
metadata:
  name: test-secret
data:
  username: c3RhdnJvcwo=
  password: Mzk1MjgkdmRnN0pi

-------

$ echo -n 'stavros' | base64
c3RhdnJvcw==
$ echo -n '39528$vdg7Jb' | base64
MWYyZDFlMmU2N2Rm
```
Run a job as follows:
```./bin/spark-submit \
      --master k8s://http://localhost:9000 \
      --deploy-mode cluster \
      --name spark-pi \
      --class org.apache.spark.examples.SparkPi \
      --conf spark.executor.instances=1 \
      --conf spark.kubernetes.container.image=skonto/spark:k8envs3 \
      --conf spark.kubernetes.driver.secretKeyRef.MY_USERNAME=test-secret:username \
      --conf spark.kubernetes.driver.secretKeyRef.My_password=test-secret:password \
      --conf spark.kubernetes.executor.secretKeyRef.MY_USERNAME=test-secret:username \
      --conf spark.kubernetes.executor.secretKeyRef.My_password=test-secret:password \
      local:///opt/spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar 10000
```

Secret loaded correctly at the driver container:
![image](https://user-images.githubusercontent.com/7945591/40174346-7fee70c8-59dd-11e8-8705-995a5472716f.png)

Also if I log into the exec container:

kubectl exec -it spark-pi-1526555613156-exec-1 bash
bash-4.4# env

> SPARK_EXECUTOR_MEMORY=1g
> SPARK_EXECUTOR_CORES=1
> LANG=C.UTF-8
> HOSTNAME=spark-pi-1526555613156-exec-1
> SPARK_APPLICATION_ID=spark-application-1526555618626
> **MY_USERNAME=stavros**
>
> JAVA_HOME=/usr/lib/jvm/java-1.8-openjdk
> KUBERNETES_PORT_443_TCP_PROTO=tcp
> KUBERNETES_PORT_443_TCP_ADDR=10.100.0.1
> JAVA_VERSION=8u151
> KUBERNETES_PORT=tcp://10.100.0.1:443
> PWD=/opt/spark/work-dir
> HOME=/root
> SPARK_LOCAL_DIRS=/var/data/spark-b569b0ae-b7ef-4f91-bcd5-0f55535d3564
> KUBERNETES_SERVICE_PORT_HTTPS=443
> KUBERNETES_PORT_443_TCP_PORT=443
> SPARK_HOME=/opt/spark
> SPARK_DRIVER_URL=spark://CoarseGrainedSchedulerspark-pi-1526555613156-driver-svc.default.svc:7078
> KUBERNETES_PORT_443_TCP=tcp://10.100.0.1:443
> SPARK_EXECUTOR_POD_IP=9.0.9.77
> TERM=xterm
> SPARK_EXECUTOR_ID=1
> SHLVL=1
> KUBERNETES_SERVICE_PORT=443
> SPARK_CONF_DIR=/opt/spark/conf
> PATH=/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/usr/lib/jvm/java-1.8-openjdk/jre/bin:/usr/lib/jvm/java-1.8-openjdk/bin
> JAVA_ALPINE_VERSION=8.151.12-r0
> KUBERNETES_SERVICE_HOST=10.100.0.1
> **My_password=39528$vdg7Jb**
> _=/usr/bin/env
>

Author: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>

Closes #21317 from skonto/k8s-fix-env-secrets.
2018-05-31 14:28:33 -07:00
mcheah 6282fc64e3 [SPARK-24137][K8S] Mount local directories as empty dir volumes.
## What changes were proposed in this pull request?

Drastically improves performance and won't cause Spark applications to fail because they write too much data to the Docker image's specific file system. The file system's directories that back emptydir volumes are generally larger and more performant.

## How was this patch tested?

Has been in use via the prototype version of Kubernetes support, but lost in the transition to here.

Author: mcheah <mcheah@palantir.com>

Closes #21238 from mccheah/mount-local-dirs.
2018-05-10 11:36:41 -07:00
Devaraj K 007ae6878f [SPARK-24003][CORE] Add support to provide spark.executor.extraJavaOptions in terms of App Id and/or Executor Id's
## What changes were proposed in this pull request?

Added support to specify the 'spark.executor.extraJavaOptions' value in terms of the `{{APP_ID}}` and/or `{{EXECUTOR_ID}}`,  `{{APP_ID}}` will be replaced by Application Id and `{{EXECUTOR_ID}}` will be replaced by Executor Id while starting the executor.

## How was this patch tested?

I have verified this by checking the executor process command and gc logs. I verified the same in different deployment modes(Standalone, YARN, Mesos) client and cluster modes.

Author: Devaraj K <devaraj@apache.org>

Closes #21088 from devaraj-kavali/SPARK-24003.
2018-04-30 13:40:03 -07:00
mcheah a83ae0d9bc [SPARK-22839][K8S] Refactor to unify driver and executor pod builder APIs
## What changes were proposed in this pull request?

Breaks down the construction of driver pods and executor pods in a way that uses a common abstraction for both spark-submit creating the driver and KubernetesClusterSchedulerBackend creating the executor. Encourages more code reuse and is more legible than the older approach.

The high-level design is discussed in more detail on the JIRA ticket. This pull request is the implementation of that design with some minor changes in the implementation details.

No user-facing behavior should break as a result of this change.

## How was this patch tested?

Migrated all unit tests from the old submission steps architecture to the new architecture. Integration tests should not have to change and pass given that this shouldn't change any outward behavior.

Author: mcheah <mcheah@palantir.com>

Closes #20910 from mccheah/spark-22839-incremental.
2018-04-13 08:43:58 -07:00
Andrew Korzhuev cccaaa14ad [SPARK-23668][K8S] Add config option for passing through k8s Pod.spec.imagePullSecrets
## What changes were proposed in this pull request?

Pass through the `imagePullSecrets` option to the k8s pod in order to allow user to access private image registries.

See https://kubernetes.io/docs/tasks/configure-pod-container/pull-image-private-registry/

## How was this patch tested?

Unit tests + manual testing.

Manual testing procedure:
1. Have private image registry.
2. Spark-submit application with no `spark.kubernetes.imagePullSecret` set. Do `kubectl describe pod ...`. See the error message:
```
Error syncing pod, skipping: failed to "StartContainer" for "spark-kubernetes-driver" with ErrImagePull: "rpc error: code = 2 desc = Error: Status 400 trying to pull repository ...: \"{\\n  \\\"errors\\\" : [ {\\n    \\\"status\\\" : 400,\\n    \\\"message\\\" : \\\"Unsupported docker v1 repository request for '...'\\\"\\n  } ]\\n}\""
```
3. Create secret `kubectl create secret docker-registry ...`
4. Spark-submit with `spark.kubernetes.imagePullSecret` set to the new secret. See that deployment was successful.

Author: Andrew Korzhuev <andrew.korzhuev@klarna.com>
Author: Andrew Korzhuev <korzhuev@andrusha.me>

Closes #20811 from andrusha/spark-23668-image-pull-secrets.
2018-04-04 12:30:52 -07:00
Yinan Li fe2b7a4568 [SPARK-23285][K8S] Add a config property for specifying physical executor cores
## What changes were proposed in this pull request?

As mentioned in SPARK-23285, this PR introduces a new configuration property `spark.kubernetes.executor.cores` for specifying the physical CPU cores requested for each executor pod. This is to avoid changing the semantics of `spark.executor.cores` and `spark.task.cpus` and their role in task scheduling, task parallelism, dynamic resource allocation, etc. The new configuration property only determines the physical CPU cores available to an executor. An executor can still run multiple tasks simultaneously by using appropriate values for `spark.executor.cores` and `spark.task.cpus`.

## How was this patch tested?

Unit tests.

felixcheung srowen jiangxb1987 jerryshao mccheah foxish

Author: Yinan Li <ynli@google.com>
Author: Yinan Li <liyinan926@gmail.com>

Closes #20553 from liyinan926/master.
2018-04-02 12:20:55 -07:00
David Vogelbacher 6151f29f9f [SPARK-23825][K8S] Requesting memory + memory overhead for pod memory
## What changes were proposed in this pull request?

Kubernetes driver and executor pods should request `memory + memoryOverhead` as their resources instead of just `memory`, see https://issues.apache.org/jira/browse/SPARK-23825

## How was this patch tested?
Existing unit tests were adapted.

Author: David Vogelbacher <dvogelbacher@palantir.com>

Closes #20943 from dvogelbacher/spark-23825.
2018-04-02 12:00:37 -07:00
Ilan Filonenko f15906da15 [SPARK-22839][K8S] Remove the use of init-container for downloading remote dependencies
## What changes were proposed in this pull request?

Removal of the init-container for downloading remote dependencies. Built off of the work done by vanzin in an attempt to refactor driver/executor configuration elaborated in [this](https://issues.apache.org/jira/browse/SPARK-22839) ticket.

## How was this patch tested?

This patch was tested with unit and integration tests.

Author: Ilan Filonenko <if56@cornell.edu>

Closes #20669 from ifilonenko/remove-init-container.
2018-03-19 11:29:56 -07:00
Yinan Li 5d7c4ba4d7 [SPARK-22962][K8S] Fail fast if submission client local files are used
## What changes were proposed in this pull request?

In the Kubernetes mode, fails fast in the submission process if any submission client local dependencies are used as the use case is not supported yet.

## How was this patch tested?

Unit tests, integration tests, and manual tests.

vanzin foxish

Author: Yinan Li <liyinan926@gmail.com>

Closes #20320 from liyinan926/master.
2018-01-18 14:44:22 -08:00
gatorsmile 651f76153f [SPARK-23028] Bump master branch version to 2.4.0-SNAPSHOT
## What changes were proposed in this pull request?
This patch bumps the master branch version to `2.4.0-SNAPSHOT`.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20222 from gatorsmile/bump24.
2018-01-13 00:37:59 +08:00
Marcelo Vanzin 0b2eefb674 [SPARK-22994][K8S] Use a single image for all Spark containers.
This change allows a user to submit a Spark application on kubernetes
having to provide a single image, instead of one image for each type
of container. The image's entry point now takes an extra argument that
identifies the process that is being started.

The configuration still allows the user to provide different images
for each container type if they so desire.

On top of that, the entry point was simplified a bit to share more
code; mainly, the same env variable is used to propagate the user-defined
classpath to the different containers.

Aside from being modified to match the new behavior, the
'build-push-docker-images.sh' script was renamed to 'docker-image-tool.sh'
to more closely match its purpose; the old name was a little awkward
and now also not entirely correct, since there is a single image. It
was also moved to 'bin' since it's not necessarily an admin tool.

Docs have been updated to match the new behavior.

Tested locally with minikube.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20192 from vanzin/SPARK-22994.
2018-01-11 10:37:35 -08:00
Yinan Li 6a4206ff04 [SPARK-22998][K8S] Set missing value for SPARK_MOUNTED_CLASSPATH in the executors
## What changes were proposed in this pull request?

The environment variable `SPARK_MOUNTED_CLASSPATH` is referenced in the executor's Dockerfile, where its value is added to the classpath of the executor. However, the scheduler backend code missed setting it when creating the executor pods. This PR fixes it.

## How was this patch tested?

Unit tested.

vanzin Can you help take a look? Thanks!
foxish

Author: Yinan Li <liyinan926@gmail.com>

Closes #20193 from liyinan926/master.
2018-01-09 01:32:48 -08:00
foxish eed82a0b21 [SPARK-22992][K8S] Remove assumption of the DNS domain
## What changes were proposed in this pull request?

Remove the use of FQDN to access the driver because it assumes that it's set up in a DNS zone - `cluster.local` which is common but not ubiquitous
Note that we already access the in-cluster API server through `kubernetes.default.svc`, so, by extension, this should work as well.
The alternative is to introduce DNS zones for both of those addresses.

## How was this patch tested?
Unit tests

cc vanzin liyinan926 mridulm mccheah

Author: foxish <ramanathana@google.com>

Closes #20187 from foxish/cluster.local.
2018-01-08 13:01:45 -08:00
Yinan Li e288fc87a0 [SPARK-22953][K8S] Avoids adding duplicated secret volumes when init-container is used
## What changes were proposed in this pull request?

User-specified secrets are mounted into both the main container and init-container (when it is used) in a Spark driver/executor pod, using the `MountSecretsBootstrap`. Because `MountSecretsBootstrap` always adds new secret volumes for the secrets to the pod, the same secret volumes get added twice, one when mounting the secrets to the main container, and the other when mounting the secrets to the init-container. This PR fixes the issue by separating `MountSecretsBootstrap.mountSecrets` out into two methods: `addSecretVolumes` for adding secret volumes to a pod and `mountSecrets` for mounting secret volumes to a container, respectively. `addSecretVolumes` is only called once for each pod, whereas `mountSecrets` is called individually for the main container and the init-container (if it is used).

Ref: https://github.com/apache-spark-on-k8s/spark/issues/594.

## How was this patch tested?
Unit tested and manually tested.

vanzin This replaces https://github.com/apache/spark/pull/20148.
hex108 foxish kimoonkim

Author: Yinan Li <liyinan926@gmail.com>

Closes #20159 from liyinan926/master.
2018-01-04 15:35:20 -08:00
Marcelo Vanzin 95f9659abe [SPARK-22948][K8S] Move SparkPodInitContainer to correct package.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20156 from vanzin/SPARK-22948.
2018-01-04 15:00:09 -08:00
Yinan Li 171f6ddadc [SPARK-22757][KUBERNETES] Enable use of remote dependencies (http, s3, gcs, etc.) in Kubernetes mode
## What changes were proposed in this pull request?

This PR expands the Kubernetes mode to be able to use remote dependencies on http/https endpoints, GCS, S3, etc. It adds steps for configuring and appending the Kubernetes init-container into the driver and executor pods for downloading remote dependencies.
[Init-containers](https://kubernetes.io/docs/concepts/workloads/pods/init-containers/), as the name suggests, are containers that are run to completion before the main containers start, and are often used to perform initialization tasks prior to starting the main containers. We use init-containers to localize remote application dependencies before the driver/executors start running. The code that the init-container runs is also included. This PR also adds a step to the driver and executors for mounting user-specified secrets that may store credentials for accessing data storage, e.g., S3 and Google Cloud Storage (GCS), into the driver and executors.

## How was this patch tested?

* The patch contains unit tests which are passing.
* Manual testing: `./build/mvn -Pkubernetes clean package` succeeded.
* Manual testing of the following cases:
  * [x] Running SparkPi using container-local spark-example jar.
  * [x] Running SparkPi using container-local spark-example jar with user-specific secret mounted.
  * [x] Running SparkPi using spark-example jar hosted remotely on an https endpoint.

cc rxin felixcheung mateiz (shepherd)
k8s-big-data SIG members & contributors: mccheah foxish ash211 ssuchter varunkatta kimoonkim erikerlandson tnachen ifilonenko liyinan926
reviewers: vanzin felixcheung jiangxb1987 mridulm

Author: Yinan Li <liyinan926@gmail.com>

Closes #19954 from liyinan926/init-container.
2017-12-28 13:44:44 +09:00
foxish 0114c89d04 [SPARK-22845][SCHEDULER] Modify spark.kubernetes.allocation.batch.delay to take time instead of int
## What changes were proposed in this pull request?

Fixing configuration that was taking an int which should take time. Discussion in https://github.com/apache/spark/pull/19946#discussion_r156682354
Made the granularity milliseconds as opposed to seconds since there's a use-case for sub-second reactions to scale-up rapidly especially with dynamic allocation.

## How was this patch tested?

TODO: manual run of integration tests against this PR.
PTAL

cc/ mccheah liyinan926 kimoonkim vanzin mridulm jiangxb1987 ueshin

Author: foxish <ramanathana@google.com>

Closes #20032 from foxish/fix-time-conf.
2017-12-20 16:14:36 -08:00
foxish fb3636b482 [SPARK-22807][SCHEDULER] Remove config that says docker and replace with container
## What changes were proposed in this pull request?
Changes discussed in https://github.com/apache/spark/pull/19946#discussion_r157063535
docker -> container, since with CRI, we are not limited to running only docker images.

## How was this patch tested?
Manual testing

Author: foxish <ramanathana@google.com>

Closes #19995 from foxish/make-docker-container.
2017-12-18 11:29:32 -08:00
Yinan Li 2fe16333d5 [SPARK-22778][KUBERNETES] Added the missing service metadata for KubernetesClusterManager
## What changes were proposed in this pull request?

This PR added the missing service metadata for `KubernetesClusterManager`. Without the metadata, the service loader couldn't load `KubernetesClusterManager`, and caused the driver to fail to create a `ExternalClusterManager`, as being reported in SPARK-22778. The PR also changed the `k8s:` prefix used to `k8s://`, which is what existing Spark on k8s users are familiar and used to.

## How was this patch tested?

Manual testing verified that the fix resolved the issue in SPARK-22778.

/cc vanzin felixcheung jiangxb1987

Author: Yinan Li <liyinan926@gmail.com>

Closes #19972 from liyinan926/fix-22778.
2017-12-14 14:03:08 -08:00
Yinan Li 3f4060c340 [SPARK-22646][K8S] Spark on Kubernetes - basic submission client
This PR contains implementation of the basic submission client for the cluster mode of Spark on Kubernetes. It's step 2 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935).
This addition is covered by the [SPIP](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) vote which passed on Aug 31.

This PR and #19468 together form a MVP of Spark on Kubernetes that allows users to run Spark applications that use resources locally within the driver and executor containers on Kubernetes 1.6 and up. Some changes on pom and build/test setup are copied over from #19468 to make this PR self contained and testable.

The submission client is mainly responsible for creating the Kubernetes pod that runs the Spark driver. It follows a step-based approach to construct the driver pod, as the code under the `submit.steps` package shows. The steps are orchestrated by `DriverConfigurationStepsOrchestrator`. `Client` creates the driver pod and waits for the application to complete if it's configured to do so, which is the case by default.

This PR also contains Dockerfiles of the driver and executor images. They are included because some of the environment variables set in the code would not make sense without referring to the Dockerfiles.

* The patch contains unit tests which are passing.
* Manual testing: ./build/mvn -Pkubernetes clean package succeeded.
* It is a subset of the entire changelist hosted at http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations.
* There is integration testing enabled in the fork currently hosted by PepperData which is being moved over to RiseLAB CI.
* Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html

cc rxin felixcheung mateiz (shepherd)
k8s-big-data SIG members & contributors: mccheah foxish ash211 ssuchter varunkatta kimoonkim erikerlandson tnachen ifilonenko liyinan926

Author: Yinan Li <liyinan926@gmail.com>

Closes #19717 from liyinan926/spark-kubernetes-4.
2017-12-11 15:15:05 -08:00
Devaraj K 51066b437b [SPARK-14228][CORE][YARN] Lost executor of RPC disassociated, and occurs exception: Could not find CoarseGrainedScheduler or it has been stopped
## What changes were proposed in this pull request?
I see the two instances where the exception is occurring.

**Instance 1:**

```
17/11/10 15:49:32 ERROR util.Utils: Uncaught exception in thread driver-revive-thread
org.apache.spark.SparkException: Could not find CoarseGrainedScheduler.
        at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:160)
        at org.apache.spark.rpc.netty.Dispatcher.postOneWayMessage(Dispatcher.scala:140)
        at org.apache.spark.rpc.netty.NettyRpcEnv.send(NettyRpcEnv.scala:187)
        at org.apache.spark.rpc.netty.NettyRpcEndpointRef.send(NettyRpcEnv.scala:521)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(CoarseGrainedSchedulerBackend.scala:125)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(CoarseGrainedSchedulerBackend.scala:125)
        at scala.Option.foreach(Option.scala:257)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1$$anonfun$run$1.apply$mcV$sp(CoarseGrainedSchedulerBackend.scala:125)
        at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1344)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1.run(CoarseGrainedSchedulerBackend.scala:124)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
```

In CoarseGrainedSchedulerBackend.scala, driver-revive-thread starts with DriverEndpoint.onStart() and keeps sending the ReviveOffers messages periodically till it gets shutdown as part DriverEndpoint.onStop(). There is no proper coordination between the driver-revive-thread(shutdown) and the RpcEndpoint unregister, RpcEndpoint unregister happens first and then driver-revive-thread shuts down as part of DriverEndpoint.onStop(), In-between driver-revive-thread may try to send the ReviveOffers message which is leading to the above exception.

To fix this issue, this PR moves the shutting down of driver-revive-thread to CoarseGrainedSchedulerBackend.stop() which executes before the DriverEndpoint unregister.

**Instance 2:**

```
17/11/10 16:31:38 ERROR cluster.YarnSchedulerBackend$YarnSchedulerEndpoint: Error requesting driver to remove executor 1 for reason Executor for container container_1508535467865_0226_01_000002 exited because of a YARN event (e.g., pre-emption) and not because of an error in the running job.
org.apache.spark.SparkException: Could not find CoarseGrainedScheduler.
        at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:160)
        at org.apache.spark.rpc.netty.Dispatcher.postLocalMessage(Dispatcher.scala:135)
        at org.apache.spark.rpc.netty.NettyRpcEnv.ask(NettyRpcEnv.scala:229)
        at org.apache.spark.rpc.netty.NettyRpcEndpointRef.ask(NettyRpcEnv.scala:516)
        at org.apache.spark.rpc.RpcEndpointRef.ask(RpcEndpointRef.scala:63)
        at org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint$$anonfun$receive$1.applyOrElse(YarnSchedulerBackend.scala:269)
        at org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp(Inbox.scala:117)
        at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:205)
        at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:101)
        at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:221)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
```

Here YarnDriverEndpoint tries to send remove executor messages after the Yarn scheduler backend service stop, which is leading to the above exception. To avoid the above exception,
1) We may add a condition(which checks whether service has stopped or not) before sending executor remove message
2) Add a warn log message in onFailure case when the service is already stopped

In this PR, chosen the 2) option which adds a log message in the case of onFailure without the exception stack trace since the option 1) would need to to go through for every remove executor message.

## How was this patch tested?
I verified it manually, I don't see these exceptions with the PR changes.

Author: Devaraj K <devaraj@apache.org>

Closes #19741 from devaraj-kavali/SPARK-14228.
2017-12-06 10:39:15 -08:00
Yinan Li e9b2070ab2 [SPARK-18278][SCHEDULER] Spark on Kubernetes - Basic Scheduler Backend
## What changes were proposed in this pull request?

This is a stripped down version of the `KubernetesClusterSchedulerBackend` for Spark with the following components:
- Static Allocation of Executors
- Executor Pod Factory
- Executor Recovery Semantics

It's step 1 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935).
This addition is covered by the [SPIP vote](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) which passed on Aug 31 .

## How was this patch tested?

- The patch contains unit tests which are passing.
- Manual testing: `./build/mvn -Pkubernetes clean package` succeeded.
- It is a **subset** of the entire changelist hosted in http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations.
- There is integration testing enabled in the fork currently [hosted by PepperData](spark-k8s-jenkins.pepperdata.org:8080) which is being moved over to RiseLAB CI.
- Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html

cc rxin felixcheung mateiz (shepherd)
k8s-big-data SIG members & contributors: mccheah ash211 ssuchter varunkatta kimoonkim erikerlandson liyinan926 tnachen ifilonenko

Author: Yinan Li <liyinan926@gmail.com>
Author: foxish <ramanathana@google.com>
Author: mcheah <mcheah@palantir.com>

Closes #19468 from foxish/spark-kubernetes-3.
2017-11-28 23:02:09 -08:00