Processing math: 100%

Running Spark on Kubernetes

Spark can run on clusters managed by Kubernetes. This feature makes use of native Kubernetes scheduler that has been added to Spark.

Security

Security features like authentication are not enabled by default. When deploying a cluster that is open to the internet or an untrusted network, it’s important to secure access to the cluster to prevent unauthorized applications from running on the cluster. Please see Spark Security and the specific security sections in this doc before running Spark.

User Identity

Images built from the project provided Dockerfiles contain a default USER directive with a default UID of 185. This means that the resulting images will be running the Spark processes as this UID inside the container. Security conscious deployments should consider providing custom images with USER directives specifying their desired unprivileged UID and GID. The resulting UID should include the root group in its supplementary groups in order to be able to run the Spark executables. Users building their own images with the provided docker-image-tool.sh script can use the -u <uid> option to specify the desired UID.

Alternatively the Pod Template feature can be used to add a Security Context with a runAsUser to the pods that Spark submits. This can be used to override the USER directives in the images themselves. Please bear in mind that this requires cooperation from your users and as such may not be a suitable solution for shared environments. Cluster administrators should use Pod Security Policies if they wish to limit the users that pods may run as.

Volume Mounts

As described later in this document under Using Kubernetes Volumes Spark on K8S provides configuration options that allow for mounting certain volume types into the driver and executor pods. In particular it allows for hostPath volumes which as described in the Kubernetes documentation have known security vulnerabilities.

Cluster administrators should use Pod Security Policies to limit the ability to mount hostPath volumes appropriately for their environments.

Prerequisites

How it works

Spark cluster components

spark-submit can be directly used to submit a Spark application to a Kubernetes cluster. The submission mechanism works as follows:

Note that in the completed state, the driver pod does not use any computational or memory resources.

The driver and executor pod scheduling is handled by Kubernetes. Communication to the Kubernetes API is done via fabric8. It is possible to schedule the driver and executor pods on a subset of available nodes through a node selector using the configuration property for it. It will be possible to use more advanced scheduling hints like node/pod affinities in a future release.

Submitting Applications to Kubernetes

Docker Images

Kubernetes requires users to supply images that can be deployed into containers within pods. The images are built to be run in a container runtime environment that Kubernetes supports. Docker is a container runtime environment that is frequently used with Kubernetes. Spark (starting with version 2.3) ships with a Dockerfile that can be used for this purpose, or customized to match an individual application’s needs. It can be found in the kubernetes/dockerfiles/ directory.

Spark also ships with a bin/docker-image-tool.sh script that can be used to build and publish the Docker images to use with the Kubernetes backend.

Example usage is:

$ ./bin/docker-image-tool.sh -r <repo> -t my-tag build
$ ./bin/docker-image-tool.sh -r <repo> -t my-tag push

This will build using the projects provided default Dockerfiles. To see more options available for customising the behaviour of this tool, including providing custom Dockerfiles, please run with the -h flag.

By default bin/docker-image-tool.sh builds docker image for running JVM jobs. You need to opt-in to build additional language binding docker images.

Example usage is

# To build additional PySpark docker image
$ ./bin/docker-image-tool.sh -r <repo> -t my-tag -p ./kubernetes/dockerfiles/spark/bindings/python/Dockerfile build

# To build additional SparkR docker image
$ ./bin/docker-image-tool.sh -r <repo> -t my-tag -R ./kubernetes/dockerfiles/spark/bindings/R/Dockerfile build

Cluster Mode

To launch Spark Pi in cluster mode,

$ ./bin/spark-submit \
    --master k8s://https://<k8s-apiserver-host>:<k8s-apiserver-port> \
    --deploy-mode cluster \
    --name spark-pi \
    --class org.apache.spark.examples.SparkPi \
    --conf spark.executor.instances=5 \
    --conf spark.kubernetes.container.image=<spark-image> \
    local:///path/to/examples.jar

The Spark master, specified either via passing the --master command line argument to spark-submit or by setting spark.master in the application’s configuration, must be a URL with the format k8s://<api_server_host>:<k8s-apiserver-port>. The port must always be specified, even if it’s the HTTPS port 443. Prefixing the master string with k8s:// will cause the Spark application to launch on the Kubernetes cluster, with the API server being contacted at api_server_url. If no HTTP protocol is specified in the URL, it defaults to https. For example, setting the master to k8s://example.com:443 is equivalent to setting it to k8s://https://example.com:443, but to connect without TLS on a different port, the master would be set to k8s://http://example.com:8080.

In Kubernetes mode, the Spark application name that is specified by spark.app.name or the --name argument to spark-submit is used by default to name the Kubernetes resources created like drivers and executors. So, application names must consist of lower case alphanumeric characters, -, and . and must start and end with an alphanumeric character.

If you have a Kubernetes cluster setup, one way to discover the apiserver URL is by executing kubectl cluster-info.

$ kubectl cluster-info
Kubernetes master is running at http://127.0.0.1:6443

In the above example, the specific Kubernetes cluster can be used with spark-submit by specifying --master k8s://http://127.0.0.1:6443 as an argument to spark-submit. Additionally, it is also possible to use the authenticating proxy, kubectl proxy to communicate to the Kubernetes API.

The local proxy can be started by:

$ kubectl proxy

If the local proxy is running at localhost:8001, --master k8s://http://127.0.0.1:8001 can be used as the argument to spark-submit. Finally, notice that in the above example we specify a jar with a specific URI with a scheme of local://. This URI is the location of the example jar that is already in the Docker image.

Client Mode

Starting with Spark 2.4.0, it is possible to run Spark applications on Kubernetes in client mode. When your application runs in client mode, the driver can run inside a pod or on a physical host. When running an application in client mode, it is recommended to account for the following factors:

Client Mode Networking

Spark executors must be able to connect to the Spark driver over a hostname and a port that is routable from the Spark executors. The specific network configuration that will be required for Spark to work in client mode will vary per setup. If you run your driver inside a Kubernetes pod, you can use a headless service to allow your driver pod to be routable from the executors by a stable hostname. When deploying your headless service, ensure that the service’s label selector will only match the driver pod and no other pods; it is recommended to assign your driver pod a sufficiently unique label and to use that label in the label selector of the headless service. Specify the driver’s hostname via spark.driver.host and your spark driver’s port to spark.driver.port.

Client Mode Executor Pod Garbage Collection

If you run your Spark driver in a pod, it is highly recommended to set spark.kubernetes.driver.pod.name to the name of that pod. When this property is set, the Spark scheduler will deploy the executor pods with an OwnerReference, which in turn will ensure that once the driver pod is deleted from the cluster, all of the application’s executor pods will also be deleted. The driver will look for a pod with the given name in the namespace specified by spark.kubernetes.namespace, and an OwnerReference pointing to that pod will be added to each executor pod’s OwnerReferences list. Be careful to avoid setting the OwnerReference to a pod that is not actually that driver pod, or else the executors may be terminated prematurely when the wrong pod is deleted.

If your application is not running inside a pod, or if spark.kubernetes.driver.pod.name is not set when your application is actually running in a pod, keep in mind that the executor pods may not be properly deleted from the cluster when the application exits. The Spark scheduler attempts to delete these pods, but if the network request to the API server fails for any reason, these pods will remain in the cluster. The executor processes should exit when they cannot reach the driver, so the executor pods should not consume compute resources (cpu and memory) in the cluster after your application exits.

You may use spark.kubernetes.executor.podNamePrefix to fully control the executor pod names. When this property is set, it’s highly recommended to make it unique across all jobs in the same namespace.

Authentication Parameters

Use the exact prefix spark.kubernetes.authenticate for Kubernetes authentication parameters in client mode.

Dependency Management

If your application’s dependencies are all hosted in remote locations like HDFS or HTTP servers, they may be referred to by their appropriate remote URIs. Also, application dependencies can be pre-mounted into custom-built Docker images. Those dependencies can be added to the classpath by referencing them with local:// URIs and/or setting the SPARK_EXTRA_CLASSPATH environment variable in your Dockerfiles. The local:// scheme is also required when referring to dependencies in custom-built Docker images in spark-submit. We support dependencies from the submission client’s local file system using the file:// scheme or without a scheme (using a full path), where the destination should be a Hadoop compatible filesystem. A typical example of this using S3 is via passing the following options:

...
--packages org.apache.hadoop:hadoop-aws:3.2.2
--conf spark.kubernetes.file.upload.path=s3a://<s3-bucket>/path
--conf spark.hadoop.fs.s3a.access.key=...
--conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem
--conf spark.hadoop.fs.s3a.fast.upload=true
--conf spark.hadoop.fs.s3a.secret.key=....
--conf spark.driver.extraJavaOptions=-Divy.cache.dir=/tmp -Divy.home=/tmp
file:///full/path/to/app.jar

The app jar file will be uploaded to the S3 and then when the driver is launched it will be downloaded to the driver pod and will be added to its classpath. Spark will generate a subdir under the upload path with a random name to avoid conflicts with spark apps running in parallel. User could manage the subdirs created according to his needs.

The client scheme is supported for the application jar, and dependencies specified by properties spark.jars, spark.files and spark.archives.

Important: all client-side dependencies will be uploaded to the given path with a flat directory structure so file names must be unique otherwise files will be overwritten. Also make sure in the derived k8s image default ivy dir has the required access rights or modify the settings as above. The latter is also important if you use --packages in cluster mode.

Secret Management

Kubernetes Secrets can be used to provide credentials for a Spark application to access secured services. To mount a user-specified secret into the driver container, users can use the configuration property of the form spark.kubernetes.driver.secrets.[SecretName]=<mount path>. Similarly, the configuration property of the form spark.kubernetes.executor.secrets.[SecretName]=<mount path> can be used to mount a user-specified secret into the executor containers. Note that it is assumed that the secret to be mounted is in the same namespace as that of the driver and executor pods. For example, to mount a secret named spark-secret onto the path /etc/secrets in both the driver and executor containers, add the following options to the spark-submit command:

--conf spark.kubernetes.driver.secrets.spark-secret=/etc/secrets
--conf spark.kubernetes.executor.secrets.spark-secret=/etc/secrets

To use a secret through an environment variable use the following options to the spark-submit command:

--conf spark.kubernetes.driver.secretKeyRef.ENV_NAME=name:key
--conf spark.kubernetes.executor.secretKeyRef.ENV_NAME=name:key

Pod Template

Kubernetes allows defining pods from template files. Spark users can similarly use template files to define the driver or executor pod configurations that Spark configurations do not support. To do so, specify the spark properties spark.kubernetes.driver.podTemplateFile and spark.kubernetes.executor.podTemplateFile to point to files accessible to the spark-submit process.

--conf spark.kubernetes.driver.podTemplateFile=s3a://bucket/driver.yml
--conf spark.kubernetes.executor.podTemplateFile=s3a://bucket/executor.yml

To allow the driver pod access the executor pod template file, the file will be automatically mounted onto a volume in the driver pod when it’s created. Spark does not do any validation after unmarshalling these template files and relies on the Kubernetes API server for validation.

It is important to note that Spark is opinionated about certain pod configurations so there are values in the pod template that will always be overwritten by Spark. Therefore, users of this feature should note that specifying the pod template file only lets Spark start with a template pod instead of an empty pod during the pod-building process. For details, see the full list of pod template values that will be overwritten by spark.

Pod template files can also define multiple containers. In such cases, you can use the spark properties spark.kubernetes.driver.podTemplateContainerName and spark.kubernetes.executor.podTemplateContainerName to indicate which container should be used as a basis for the driver or executor. If not specified, or if the container name is not valid, Spark will assume that the first container in the list will be the driver or executor container.

Using Kubernetes Volumes

Users can mount the following types of Kubernetes volumes into the driver and executor pods:

NB: Please see the Security section of this document for security issues related to volume mounts.

To mount a volume of any of the types above into the driver pod, use the following configuration property:

--conf spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.path=<mount path>
--conf spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.readOnly=<true|false>
--conf spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.subPath=<mount subPath>

Specifically, VolumeType can be one of the following values: hostPath, emptyDir, nfs and persistentVolumeClaim. VolumeName is the name you want to use for the volume under the volumes field in the pod specification.

Each supported type of volumes may have some specific configuration options, which can be specified using configuration properties of the following form:

spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].options.[OptionName]=<value>

For example, the server and path of a nfs with volume name images can be specified using the following properties:

spark.kubernetes.driver.volumes.nfs.images.options.server=example.com
spark.kubernetes.driver.volumes.nfs.images.options.path=/data

And, the claim name of a persistentVolumeClaim with volume name checkpointpvc can be specified using the following property:

spark.kubernetes.driver.volumes.persistentVolumeClaim.checkpointpvc.options.claimName=check-point-pvc-claim

The configuration properties for mounting volumes into the executor pods use prefix spark.kubernetes.executor. instead of spark.kubernetes.driver..

For example, you can mount a dynamically-created persistent volume claim per executor by using OnDemand as a claim name and storageClass and sizeLimit options like the following. This is useful in case of Dynamic Allocation.

spark.kubernetes.executor.volumes.persistentVolumeClaim.data.options.claimName=OnDemand
spark.kubernetes.executor.volumes.persistentVolumeClaim.data.options.storageClass=gp
spark.kubernetes.executor.volumes.persistentVolumeClaim.data.options.sizeLimit=500Gi
spark.kubernetes.executor.volumes.persistentVolumeClaim.data.mount.path=/data
spark.kubernetes.executor.volumes.persistentVolumeClaim.data.mount.readOnly=false

For a complete list of available options for each supported type of volumes, please refer to the Spark Properties section below.

Local Storage

Spark supports using volumes to spill data during shuffles and other operations. To use a volume as local storage, the volume’s name should starts with spark-local-dir-, for example:

--conf spark.kubernetes.driver.volumes.[VolumeType].spark-local-dir-[VolumeName].mount.path=<mount path>
--conf spark.kubernetes.driver.volumes.[VolumeType].spark-local-dir-[VolumeName].mount.readOnly=false

Specifically, you can use persistent volume claims if the jobs require large shuffle and sorting operations in executors.

spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.options.claimName=OnDemand
spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.options.storageClass=gp
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

If no volume is set as local storage, Spark uses temporary scratch space to spill data to disk during shuffles and other operations. When using Kubernetes as the resource manager the pods will be created with an emptyDir volume mounted for each directory listed in spark.local.dir or the environment variable SPARK_LOCAL_DIRS . If no directories are explicitly specified then a default directory is created and configured appropriately.

emptyDir volumes use the ephemeral storage feature of Kubernetes and do not persist beyond the life of the pod.

Using RAM for local storage

emptyDir volumes use the nodes backing storage for ephemeral storage by default, this behaviour may not be appropriate for some compute environments. For example if you have diskless nodes with remote storage mounted over a network, having lots of executors doing IO to this remote storage may actually degrade performance.

In this case it may be desirable to set spark.kubernetes.local.dirs.tmpfs=true in your configuration which will cause the emptyDir volumes to be configured as tmpfs i.e. RAM backed volumes. When configured like this Spark’s local storage usage will count towards your pods memory usage therefore you may wish to increase your memory requests by increasing the value of spark.{driver,executor}.memoryOverheadFactor as appropriate.

Introspection and Debugging

These are the different ways in which you can investigate a running/completed Spark application, monitor progress, and take actions.

Accessing Logs

Logs can be accessed using the Kubernetes API and the kubectl CLI. When a Spark application is running, it’s possible to stream logs from the application using:

$ kubectl -n=<namespace> logs -f <driver-pod-name>

The same logs can also be accessed through the Kubernetes dashboard if installed on the cluster.

Accessing Driver UI

The UI associated with any application can be accessed locally using kubectl port-forward.

$ kubectl port-forward <driver-pod-name> 4040:4040

Then, the Spark driver UI can be accessed on http://localhost:4040.

Debugging

There may be several kinds of failures. If the Kubernetes API server rejects the request made from spark-submit, or the connection is refused for a different reason, the submission logic should indicate the error encountered. However, if there are errors during the running of the application, often, the best way to investigate may be through the Kubernetes CLI.

To get some basic information about the scheduling decisions made around the driver pod, you can run:

$ kubectl describe pod <spark-driver-pod>

If the pod has encountered a runtime error, the status can be probed further using:

$ kubectl logs <spark-driver-pod>

Status and logs of failed executor pods can be checked in similar ways. Finally, deleting the driver pod will clean up the entire spark application, including all executors, associated service, etc. The driver pod can be thought of as the Kubernetes representation of the Spark application.

Kubernetes Features

Configuration File

Your Kubernetes config file typically lives under .kube/config in your home directory or in a location specified by the KUBECONFIG environment variable. Spark on Kubernetes will attempt to use this file to do an initial auto-configuration of the Kubernetes client used to interact with the Kubernetes cluster. A variety of Spark configuration properties are provided that allow further customising the client configuration e.g. using an alternative authentication method.

Contexts

Kubernetes configuration files can contain multiple contexts that allow for switching between different clusters and/or user identities. By default Spark on Kubernetes will use your current context (which can be checked by running kubectl config current-context) when doing the initial auto-configuration of the Kubernetes client.

In order to use an alternative context users can specify the desired context via the Spark configuration property spark.kubernetes.context e.g. spark.kubernetes.context=minikube.

Namespaces

Kubernetes has the concept of namespaces. Namespaces are ways to divide cluster resources between multiple users (via resource quota). Spark on Kubernetes can use namespaces to launch Spark applications. This can be made use of through the spark.kubernetes.namespace configuration.

Kubernetes allows using ResourceQuota to set limits on resources, number of objects, etc on individual namespaces. Namespaces and ResourceQuota can be used in combination by administrator to control sharing and resource allocation in a Kubernetes cluster running Spark applications.

RBAC

In Kubernetes clusters with RBAC enabled, users can configure Kubernetes RBAC roles and service accounts used by the various Spark on Kubernetes components to access the Kubernetes API server.

The Spark driver pod uses a Kubernetes service account to access the Kubernetes API server to create and watch executor pods. The service account used by the driver pod must have the appropriate permission for the driver to be able to do its work. Specifically, at minimum, the service account must be granted a Role or ClusterRole that allows driver pods to create pods and services. By default, the driver pod is automatically assigned the default service account in the namespace specified by spark.kubernetes.namespace, if no service account is specified when the pod gets created.

Depending on the version and setup of Kubernetes deployed, this default service account may or may not have the role that allows driver pods to create pods and services under the default Kubernetes RBAC policies. Sometimes users may need to specify a custom service account that has the right role granted. Spark on Kubernetes supports specifying a custom service account to be used by the driver pod through the configuration property spark.kubernetes.authenticate.driver.serviceAccountName=<service account name>. For example, to make the driver pod use the spark service account, a user simply adds the following option to the spark-submit command:

--conf spark.kubernetes.authenticate.driver.serviceAccountName=spark

To create a custom service account, a user can use the kubectl create serviceaccount command. For example, the following command creates a service account named spark:

$ kubectl create serviceaccount spark

To grant a service account a Role or ClusterRole, a RoleBinding or ClusterRoleBinding is needed. To create a RoleBinding or ClusterRoleBinding, a user can use the kubectl create rolebinding (or clusterrolebinding for ClusterRoleBinding) command. For example, the following command creates an edit ClusterRole in the default namespace and grants it to the spark service account created above:

$ kubectl create clusterrolebinding spark-role --clusterrole=edit --serviceaccount=default:spark --namespace=default

Note that a Role can only be used to grant access to resources (like pods) within a single namespace, whereas a ClusterRole can be used to grant access to cluster-scoped resources (like nodes) as well as namespaced resources (like pods) across all namespaces. For Spark on Kubernetes, since the driver always creates executor pods in the same namespace, a Role is sufficient, although users may use a ClusterRole instead. For more information on RBAC authorization and how to configure Kubernetes service accounts for pods, please refer to Using RBAC Authorization and Configure Service Accounts for Pods.

Spark Application Management

Kubernetes provides simple application management via the spark-submit CLI tool in cluster mode. Users can kill a job by providing the submission ID that is printed when submitting their job. The submission ID follows the format namespace:driver-pod-name. If user omits the namespace then the namespace set in current k8s context is used. For example if user has set a specific namespace as follows kubectl config set-context minikube --namespace=spark then the spark namespace will be used by default. On the other hand, if there is no namespace added to the specific context then all namespaces will be considered by default. That means operations will affect all Spark applications matching the given submission ID regardless of namespace. Moreover, spark-submit for application management uses the same backend code that is used for submitting the driver, so the same properties like spark.kubernetes.context etc., can be re-used.

For example:

$ spark-submit --kill spark:spark-pi-1547948636094-driver --master k8s://https://192.168.2.8:8443

Users also can list the application status by using the --status flag:

$ spark-submit --status spark:spark-pi-1547948636094-driver --master  k8s://https://192.168.2.8:8443

Both operations support glob patterns. For example user can run:

$ spark-submit --kill spark:spark-pi* --master  k8s://https://192.168.2.8:8443

The above will kill all application with the specific prefix.

User can specify the grace period for pod termination via the spark.kubernetes.appKillPodDeletionGracePeriod property, using --conf as means to provide it (default value for all K8s pods is 30 secs).

Future Work

There are several Spark on Kubernetes features that are currently being worked on or planned to be worked on. Those features are expected to eventually make it into future versions of the spark-kubernetes integration.

Some of these include:

Configuration

See the configuration page for information on Spark configurations. The following configurations are specific to Spark on Kubernetes.

Spark Properties

Property NameDefaultMeaningSince Version
spark.kubernetes.context (none) The context from the user Kubernetes configuration file used for the initial auto-configuration of the Kubernetes client library. When not specified then the users current context is used. NB: Many of the auto-configured settings can be overridden by the use of other Spark configuration properties e.g. spark.kubernetes.namespace. 3.0.0
spark.kubernetes.driver.master https://kubernetes.default.svc The internal Kubernetes master (API server) address to be used for driver to request executors. 3.0.0
spark.kubernetes.namespace default The namespace that will be used for running the driver and executor pods. 2.3.0
spark.kubernetes.container.image (none) Container image to use for the Spark application. This is usually of the form example.com/repo/spark:v1.0.0. This configuration is required and must be provided by the user, unless explicit images are provided for each different container type. 2.3.0
spark.kubernetes.driver.container.image (value of spark.kubernetes.container.image) Custom container image to use for the driver. 2.3.0
spark.kubernetes.executor.container.image (value of spark.kubernetes.container.image) Custom container image to use for executors. 2.3.0
spark.kubernetes.container.image.pullPolicy IfNotPresent Container image pull policy used when pulling images within Kubernetes. Valid values are Always, Never, and IfNotPresent. 2.3.0
spark.kubernetes.container.image.pullSecrets Comma separated list of Kubernetes secrets used to pull images from private image registries. 2.4.0
spark.kubernetes.allocation.batch.size 5 Number of pods to launch at once in each round of executor pod allocation. 2.3.0
spark.kubernetes.allocation.batch.delay 1s Time to wait between each round of executor pod allocation. Specifying values less than 1 second may lead to excessive CPU usage on the spark driver. 2.3.0
spark.kubernetes.authenticate.submission.caCertFile (none) Path to the CA cert file for connecting to the Kubernetes API server over TLS when starting the driver. This file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use spark.kubernetes.authenticate.caCertFile instead. 2.3.0
spark.kubernetes.authenticate.submission.clientKeyFile (none) Path to the client key file for authenticating against the Kubernetes API server when starting the driver. This file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use spark.kubernetes.authenticate.clientKeyFile instead. 2.3.0
spark.kubernetes.authenticate.submission.clientCertFile (none) Path to the client cert file for authenticating against the Kubernetes API server when starting the driver. This file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use spark.kubernetes.authenticate.clientCertFile instead. 2.3.0
spark.kubernetes.authenticate.submission.oauthToken (none) OAuth token to use when authenticating against the Kubernetes API server when starting the driver. Note that unlike the other authentication options, this is expected to be the exact string value of the token to use for the authentication. In client mode, use spark.kubernetes.authenticate.oauthToken instead. 2.3.0
spark.kubernetes.authenticate.submission.oauthTokenFile (none) Path to the OAuth token file containing the token to use when authenticating against the Kubernetes API server when starting the driver. This file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use spark.kubernetes.authenticate.oauthTokenFile instead. 2.3.0
spark.kubernetes.authenticate.driver.caCertFile (none) Path to the CA cert file for connecting to the Kubernetes API server over TLS from the driver pod when requesting executors. This file must be located on the submitting machine's disk, and will be uploaded to the driver pod. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use spark.kubernetes.authenticate.caCertFile instead. 2.3.0
spark.kubernetes.authenticate.driver.clientKeyFile (none) Path to the client key file for authenticating against the Kubernetes API server from the driver pod when requesting executors. This file must be located on the submitting machine's disk, and will be uploaded to the driver pod as a Kubernetes secret. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use spark.kubernetes.authenticate.clientKeyFile instead. 2.3.0
spark.kubernetes.authenticate.driver.clientCertFile (none) Path to the client cert file for authenticating against the Kubernetes API server from the driver pod when requesting executors. This file must be located on the submitting machine's disk, and will be uploaded to the driver pod as a Kubernetes secret. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use spark.kubernetes.authenticate.clientCertFile instead. 2.3.0
spark.kubernetes.authenticate.driver.oauthToken (none) OAuth token to use when authenticating against the Kubernetes API server from the driver pod when requesting executors. Note that unlike the other authentication options, this must be the exact string value of the token to use for the authentication. This token value is uploaded to the driver pod as a Kubernetes secret. In client mode, use spark.kubernetes.authenticate.oauthToken instead. 2.3.0
spark.kubernetes.authenticate.driver.oauthTokenFile (none) Path to the OAuth token file containing the token to use when authenticating against the Kubernetes API server from the driver pod when requesting executors. Note that unlike the other authentication options, this file must contain the exact string value of the token to use for the authentication. This token value is uploaded to the driver pod as a secret. In client mode, use spark.kubernetes.authenticate.oauthTokenFile instead. 2.3.0
spark.kubernetes.authenticate.driver.mounted.caCertFile (none) Path to the CA cert file for connecting to the Kubernetes API server over TLS from the driver pod when requesting executors. This path must be accessible from the driver pod. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use spark.kubernetes.authenticate.caCertFile instead. 2.3.0
spark.kubernetes.authenticate.driver.mounted.clientKeyFile (none) Path to the client key file for authenticating against the Kubernetes API server from the driver pod when requesting executors. This path must be accessible from the driver pod. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use spark.kubernetes.authenticate.clientKeyFile instead. 2.3.0
spark.kubernetes.authenticate.driver.mounted.clientCertFile (none) Path to the client cert file for authenticating against the Kubernetes API server from the driver pod when requesting executors. This path must be accessible from the driver pod. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use spark.kubernetes.authenticate.clientCertFile instead. 2.3.0
spark.kubernetes.authenticate.driver.mounted.oauthTokenFile (none) Path to the file containing the OAuth token to use when authenticating against the Kubernetes API server from the driver pod when requesting executors. This path must be accessible from the driver pod. Note that unlike the other authentication options, this file must contain the exact string value of the token to use for the authentication. In client mode, use spark.kubernetes.authenticate.oauthTokenFile instead. 2.3.0
spark.kubernetes.authenticate.driver.serviceAccountName default Service account that is used when running the driver pod. The driver pod uses this service account when requesting executor pods from the API server. Note that this cannot be specified alongside a CA cert file, client key file, client cert file, and/or OAuth token. In client mode, use spark.kubernetes.authenticate.serviceAccountName instead. 2.3.0
spark.kubernetes.authenticate.executor.serviceAccountName (value of spark.kubernetes.authenticate.driver.serviceAccountName) Service account that is used when running the executor pod. If this parameter is not setup, the fallback logic will use the driver's service account. 3.1.0
spark.kubernetes.authenticate.caCertFile (none) In client mode, path to the CA cert file for connecting to the Kubernetes API server over TLS when requesting executors. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). 2.4.0
spark.kubernetes.authenticate.clientKeyFile (none) In client mode, path to the client key file for authenticating against the Kubernetes API server when requesting executors. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). 2.4.0
spark.kubernetes.authenticate.clientCertFile (none) In client mode, path to the client cert file for authenticating against the Kubernetes API server when requesting executors. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). 2.4.0
spark.kubernetes.authenticate.oauthToken (none) In client mode, the OAuth token to use when authenticating against the Kubernetes API server when requesting executors. Note that unlike the other authentication options, this must be the exact string value of the token to use for the authentication. 2.4.0
spark.kubernetes.authenticate.oauthTokenFile (none) In client mode, path to the file containing the OAuth token to use when authenticating against the Kubernetes API server when requesting executors. 2.4.0
spark.kubernetes.driver.label.[LabelName] (none) Add the label specified by LabelName to the driver pod. For example, spark.kubernetes.driver.label.something=true. Note that Spark also adds its own labels to the driver pod for bookkeeping purposes. 2.3.0
spark.kubernetes.driver.annotation.[AnnotationName] (none) Add the Kubernetes annotation specified by AnnotationName to the driver pod. For example, spark.kubernetes.driver.annotation.something=true. 2.3.0
spark.kubernetes.driver.service.annotation.[AnnotationName] (none) Add the Kubernetes annotation specified by AnnotationName to the driver service. For example, spark.kubernetes.driver.service.annotation.something=true. 3.0.0
spark.kubernetes.executor.label.[LabelName] (none) Add the label specified by LabelName to the executor pods. For example, spark.kubernetes.executor.label.something=true. Note that Spark also adds its own labels to the executor pod for bookkeeping purposes. 2.3.0
spark.kubernetes.executor.annotation.[AnnotationName] (none) Add the Kubernetes annotation specified by AnnotationName to the executor pods. For example, spark.kubernetes.executor.annotation.something=true. 2.3.0
spark.kubernetes.driver.pod.name (none) Name of the driver pod. In cluster mode, if this is not set, the driver pod name is set to "spark.app.name" suffixed by the current timestamp to avoid name conflicts. In client mode, if your application is running inside a pod, it is highly recommended to set this to the name of the pod your driver is running in. Setting this value in client mode allows the driver to become the owner of its executor pods, which in turn allows the executor pods to be garbage collected by the cluster. 2.3.0
spark.kubernetes.executor.podNamePrefix (none) Prefix to use in front of the executor pod names. It must conform the rules defined by the Kubernetes DNS Label Names. The prefix will be used to generate executor pod names in the form of podNamePrefixexecid, where the `id` is a positive int value, so the length of the `podNamePrefix` needs to be less than or equal to 47(= 63 - 10 - 6). 2.3.0
spark.kubernetes.executor.lostCheck.maxAttempts 10 Number of times that the driver will try to ascertain the loss reason for a specific executor. The loss reason is used to ascertain whether the executor failure is due to a framework or an application error which in turn decides whether the executor is removed and replaced, or placed into a failed state for debugging. 2.3.0
spark.kubernetes.submission.waitAppCompletion true In cluster mode, whether to wait for the application to finish before exiting the launcher process. When changed to false, the launcher has a "fire-and-forget" behavior when launching the Spark job. 2.3.0
spark.kubernetes.report.interval 1s Interval between reports of the current Spark job status in cluster mode. 2.3.0
spark.kubernetes.executor.apiPollingInterval 30s Interval between polls against the Kubernetes API server to inspect the state of executors. 2.4.0
spark.kubernetes.driver.request.cores (none) Specify the cpu request for the driver pod. Values conform to the Kubernetes convention. Example values include 0.1, 500m, 1.5, 5, etc., with the definition of cpu units documented in CPU units. This takes precedence over spark.driver.cores for specifying the driver pod cpu request if set. 3.0.0
spark.kubernetes.driver.limit.cores (none) Specify a hard cpu limit for the driver pod. 2.3.0
spark.kubernetes.executor.request.cores (none) Specify the cpu request for each executor pod. Values conform to the Kubernetes convention. Example values include 0.1, 500m, 1.5, 5, etc., with the definition of cpu units documented in CPU units. This is distinct from spark.executor.cores: it is only used and takes precedence over spark.executor.cores for specifying the executor pod cpu request if set. Task parallelism, e.g., number of tasks an executor can run concurrently is not affected by this. 2.4.0
spark.kubernetes.executor.limit.cores (none) Specify a hard cpu limit for each executor pod launched for the Spark Application. 2.3.0
spark.kubernetes.node.selector.[labelKey] (none) Adds to the node selector of the driver pod and executor pods, with key labelKey and the value as the configuration's value. For example, setting spark.kubernetes.node.selector.identifier to myIdentifier will result in the driver pod and executors having a node selector with key identifier and value myIdentifier. Multiple node selector keys can be added by setting multiple configurations with this prefix. 2.3.0
spark.kubernetes.driver.node.selector.[labelKey] (none) Adds to the driver node selector of the driver pod, with key labelKey and the value as the configuration's value. For example, setting spark.kubernetes.driver.node.selector.identifier to myIdentifier will result in the driver pod having a node selector with key identifier and value myIdentifier. Multiple driver node selector keys can be added by setting multiple configurations with this prefix. 3.3.0
spark.kubernetes.executor.node.selector.[labelKey] (none) Adds to the executor node selector of the executor pods, with key labelKey and the value as the configuration's value. For example, setting spark.kubernetes.executor.node.selector.identifier to myIdentifier will result in the executors having a node selector with key identifier and value myIdentifier. Multiple executor node selector keys can be added by setting multiple configurations with this prefix. 3.3.0
spark.kubernetes.driverEnv.[EnvironmentVariableName] (none) Add the environment variable specified by EnvironmentVariableName to the Driver process. The user can specify multiple of these to set multiple environment variables. 2.3.0
spark.kubernetes.driver.secrets.[SecretName] (none) Add the Kubernetes Secret named SecretName to the driver pod on the path specified in the value. For example, spark.kubernetes.driver.secrets.spark-secret=/etc/secrets. 2.3.0
spark.kubernetes.executor.secrets.[SecretName] (none) Add the Kubernetes Secret named SecretName to the executor pod on the path specified in the value. For example, spark.kubernetes.executor.secrets.spark-secret=/etc/secrets. 2.3.0
spark.kubernetes.driver.secretKeyRef.[EnvName] (none) Add as an environment variable to the driver container with name EnvName (case sensitive), the value referenced by key key in the data of the referenced Kubernetes Secret. For example, spark.kubernetes.driver.secretKeyRef.ENV_VAR=spark-secret:key. 2.4.0
spark.kubernetes.executor.secretKeyRef.[EnvName] (none) Add as an environment variable to the executor container with name EnvName (case sensitive), the value referenced by key key in the data of the referenced Kubernetes Secret. For example, spark.kubernetes.executor.secrets.ENV_VAR=spark-secret:key. 2.4.0
spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.path (none) Add the Kubernetes Volume named VolumeName of the VolumeType type to the driver pod on the path specified in the value. For example, spark.kubernetes.driver.volumes.persistentVolumeClaim.checkpointpvc.mount.path=/checkpoint. 2.4.0
spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.subPath (none) Specifies a subpath to be mounted from the volume into the driver pod. spark.kubernetes.driver.volumes.persistentVolumeClaim.checkpointpvc.mount.subPath=checkpoint. 3.0.0
spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.readOnly (none) Specify if the mounted volume is read only or not. For example, spark.kubernetes.driver.volumes.persistentVolumeClaim.checkpointpvc.mount.readOnly=false. 2.4.0
spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].options.[OptionName] (none) Configure Kubernetes Volume options passed to the Kubernetes with OptionName as key having specified value, must conform with Kubernetes option format. For example, spark.kubernetes.driver.volumes.persistentVolumeClaim.checkpointpvc.options.claimName=spark-pvc-claim. 2.4.0
spark.kubernetes.executor.volumes.[VolumeType].[VolumeName].mount.path (none) Add the Kubernetes Volume named VolumeName of the VolumeType type to the executor pod on the path specified in the value. For example, spark.kubernetes.executor.volumes.persistentVolumeClaim.checkpointpvc.mount.path=/checkpoint. 2.4.0
spark.kubernetes.executor.volumes.[VolumeType].[VolumeName].mount.subPath (none) Specifies a subpath to be mounted from the volume into the executor pod. spark.kubernetes.executor.volumes.persistentVolumeClaim.checkpointpvc.mount.subPath=checkpoint. 3.0.0
spark.kubernetes.executor.volumes.[VolumeType].[VolumeName].mount.readOnly false Specify if the mounted volume is read only or not. For example, spark.kubernetes.executor.volumes.persistentVolumeClaim.checkpointpvc.mount.readOnly=false. 2.4.0
spark.kubernetes.executor.volumes.[VolumeType].[VolumeName].options.[OptionName] (none) Configure Kubernetes Volume options passed to the Kubernetes with OptionName as key having specified value. For example, spark.kubernetes.executor.volumes.persistentVolumeClaim.checkpointpvc.options.claimName=spark-pvc-claim. 2.4.0
spark.kubernetes.local.dirs.tmpfs false Configure the emptyDir volumes used to back SPARK_LOCAL_DIRS within the Spark driver and executor pods to use tmpfs backing i.e. RAM. See Local Storage earlier on this page for more discussion of this. 3.0.0
spark.kubernetes.memoryOverheadFactor 0.1 This sets the Memory Overhead Factor that will allocate memory to non-JVM memory, which includes off-heap memory allocations, non-JVM tasks, various systems processes, and tmpfs-based local directories when spark.kubernetes.local.dirs.tmpfs is true. For JVM-based jobs this value will default to 0.10 and 0.40 for non-JVM jobs. This is done as non-JVM tasks need more non-JVM heap space and such tasks commonly fail with "Memory Overhead Exceeded" errors. This preempts this error with a higher default. This will be overridden by the value set by spark.driver.memoryOverheadFactor and spark.executor.memoryOverheadFactor explicitly. 2.4.0
spark.kubernetes.pyspark.pythonVersion "3" This sets the major Python version of the docker image used to run the driver and executor containers. It can be only "3". This configuration was deprecated from Spark 3.1.0, and is effectively no-op. Users should set 'spark.pyspark.python' and 'spark.pyspark.driver.python' configurations or 'PYSPARK_PYTHON' and 'PYSPARK_DRIVER_PYTHON' environment variables. 2.4.0
spark.kubernetes.kerberos.krb5.path (none) Specify the local location of the krb5.conf file to be mounted on the driver and executors for Kerberos interaction. It is important to note that the KDC defined needs to be visible from inside the containers. 3.0.0
spark.kubernetes.kerberos.krb5.configMapName (none) Specify the name of the ConfigMap, containing the krb5.conf file, to be mounted on the driver and executors for Kerberos interaction. The KDC defined needs to be visible from inside the containers. The ConfigMap must also be in the same namespace of the driver and executor pods. 3.0.0
spark.kubernetes.hadoop.configMapName (none) Specify the name of the ConfigMap, containing the HADOOP_CONF_DIR files, to be mounted on the driver and executors for custom Hadoop configuration. 3.0.0
spark.kubernetes.kerberos.tokenSecret.name (none) Specify the name of the secret where your existing delegation tokens are stored. This removes the need for the job user to provide any kerberos credentials for launching a job. 3.0.0
spark.kubernetes.kerberos.tokenSecret.itemKey (none) Specify the item key of the data where your existing delegation tokens are stored. This removes the need for the job user to provide any kerberos credentials for launching a job. 3.0.0
spark.kubernetes.driver.podTemplateFile (none) Specify the local file that contains the driver pod template. For example spark.kubernetes.driver.podTemplateFile=/path/to/driver-pod-template.yaml 3.0.0
spark.kubernetes.driver.podTemplateContainerName (none) Specify the container name to be used as a basis for the driver in the given pod template. For example spark.kubernetes.driver.podTemplateContainerName=spark-driver 3.0.0
spark.kubernetes.executor.podTemplateFile (none) Specify the local file that contains the executor pod template. For example spark.kubernetes.executor.podTemplateFile=/path/to/executor-pod-template.yaml 3.0.0
spark.kubernetes.executor.podTemplateContainerName (none) Specify the container name to be used as a basis for the executor in the given pod template. For example spark.kubernetes.executor.podTemplateContainerName=spark-executor 3.0.0
spark.kubernetes.executor.deleteOnTermination true Specify whether executor pods should be deleted in case of failure or normal termination. 3.0.0
spark.kubernetes.executor.checkAllContainers false Specify whether executor pods should be check all containers (including sidecars) or only the executor container when determining the pod status. 3.1.0
spark.kubernetes.submission.connectionTimeout 10000 Connection timeout in milliseconds for the kubernetes client to use for starting the driver. 3.0.0
spark.kubernetes.submission.requestTimeout 10000 Request timeout in milliseconds for the kubernetes client to use for starting the driver. 3.0.0
spark.kubernetes.driver.connectionTimeout 10000 Connection timeout in milliseconds for the kubernetes client in driver to use when requesting executors. 3.0.0
spark.kubernetes.driver.requestTimeout 10000 Request timeout in milliseconds for the kubernetes client in driver to use when requesting executors. 3.0.0
spark.kubernetes.appKillPodDeletionGracePeriod (none) Specify the grace period in seconds when deleting a Spark application using spark-submit. 3.0.0
spark.kubernetes.dynamicAllocation.deleteGracePeriod 5s How long to wait for executors to shut down gracefully before a forceful kill. 3.0.0
spark.kubernetes.file.upload.path (none) Path to store files at the spark submit side in cluster mode. For example: spark.kubernetes.file.upload.path=s3a://<s3-bucket>/path File should specified as file://path/to/file or absolute path. 3.0.0
spark.kubernetes.executor.decommissionLabel (none) Label to be applied to pods which are exiting or being decommissioned. Intended for use with pod disruption budgets, deletion costs, and similar. 3.3.0
spark.kubernetes.executor.decommissionLabelValue (none) Value to be applied with the label when spark.kubernetes.executor.decommissionLabel is enabled. 3.3.0
spark.kubernetes.executor.scheduler.name (none) Specify the scheduler name for each executor pod. 3.0.0
spark.kubernetes.driver.scheduler.name (none) Specify the scheduler name for driver pod. 3.3.0
spark.kubernetes.scheduler.name (none) Specify the scheduler name for driver and executor pods. If `spark.kubernetes.driver.scheduler.name` or `spark.kubernetes.executor.scheduler.name` is set, will override this. 3.3.0
spark.kubernetes.configMap.maxSize 1572864 Max size limit for a config map. This is configurable as per limit on k8s server end. 3.1.0
spark.kubernetes.executor.missingPodDetectDelta 30s When a registered executor's POD is missing from the Kubernetes API server's polled list of PODs then this delta time is taken as the accepted time difference between the registration time and the time of the polling. After this time the POD is considered missing from the cluster and the executor will be removed. 3.1.1
spark.kubernetes.decommission.script /opt/decom.sh The location of the script to use for graceful decommissioning. 3.2.0
spark.kubernetes.driver.service.deleteOnTermination true If true, driver service will be deleted on Spark application termination. If false, it will be cleaned up when the driver pod is deletion. 3.2.0
spark.kubernetes.driver.ownPersistentVolumeClaim false If true, driver pod becomes the owner of on-demand persistent volume claims instead of the executor pods 3.2.0
spark.kubernetes.driver.reusePersistentVolumeClaim false If true, driver pod tries to reuse driver-owned on-demand persistent volume claims of the deleted executor pods if exists. This can be useful to reduce executor pod creation delay by skipping persistent volume creations. Note that a pod in `Terminating` pod status is not a deleted pod by definition and its resources including persistent volume claims are not reusable yet. Spark will create new persistent volume claims when there exists no reusable one. In other words, the total number of persistent volume claims can be larger than the number of running executors sometimes. This config requires spark.kubernetes.driver.ownPersistentVolumeClaim=true. 3.2.0
spark.kubernetes.executor.disableConfigMap false If true, disable ConfigMap creation for executors. 3.2.0
spark.kubernetes.driver.pod.featureSteps (none) Class names of an extra driver pod feature step implementing `KubernetesFeatureConfigStep`. This is a developer API. Comma separated. Runs after all of Spark internal feature steps. Since 3.3.0, your driver feature step can implement `KubernetesDriverCustomFeatureConfigStep` where the driver config is also available. 3.2.0
spark.kubernetes.executor.pod.featureSteps (none) Class names of an extra executor pod feature step implementing `KubernetesFeatureConfigStep`. This is a developer API. Comma separated. Runs after all of Spark internal feature steps. Since 3.3.0, your executor feature step can implement `KubernetesExecutorCustomFeatureConfigStep` where the executor config is also available. 3.2.0
spark.kubernetes.allocation.maxPendingPods Int.MaxValue Maximum number of pending PODs allowed during executor allocation for this application. Those newly requested executors which are unknown by Kubernetes yet are also counted into this limit as they will change into pending PODs by time. This limit is independent from the resource profiles as it limits the sum of all allocation for all the used resource profiles. 3.2.0
spark.kubernetes.allocation.pods.allocator direct Allocator to use for pods. Possible values are direct (the default) and statefulset, or a full class name of a class implementing `AbstractPodsAllocator`. Future version may add Job or replicaset. This is a developer API and may change or be removed at anytime. 3.3.0
spark.kubernetes.allocation.executor.timeout 600s Time to wait before a newly created executor POD request, which does not reached the POD pending state yet, considered timedout and will be deleted. 3.1.0
spark.kubernetes.allocation.driver.readinessTimeout 1s Time to wait for driver pod to get ready before creating executor pods. This wait only happens on application start. If timeout happens, executor pods will still be created. 3.1.3
spark.kubernetes.executor.enablePollingWithResourceVersion false If true, `resourceVersion` is set with `0` during invoking pod listing APIs in order to allow API Server-side caching. This should be used carefully. 3.3.0
spark.kubernetes.executor.rollInterval 0s Interval between executor roll operations. It's disabled by default with `0s`. 3.3.0
spark.kubernetes.executor.minTasksPerExecutorBeforeRolling 0 The minimum number of tasks per executor before rolling. Spark will not roll executors whose total number of tasks is smaller than this configuration. The default value is zero. 3.3.0
spark.kubernetes.executor.rollPolicy OUTLIER Executor roll policy: Valid values are ID, ADD_TIME, TOTAL_GC_TIME, TOTAL_DURATION, FAILED_TASKS, and OUTLIER (default). When executor roll happens, Spark uses this policy to choose an executor and decommission it. The built-in policies are based on executor summary and newly started executors are protected by spark.kubernetes.executor.minTasksPerExecutorBeforeRolling. ID policy chooses an executor with the smallest executor ID. ADD_TIME policy chooses an executor with the smallest add-time. TOTAL_GC_TIME policy chooses an executor with the biggest total task GC time. TOTAL_DURATION policy chooses an executor with the biggest total task time. AVERAGE_DURATION policy chooses an executor with the biggest average task time. FAILED_TASKS policy chooses an executor with the most number of failed tasks. OUTLIER policy chooses an executor with outstanding statistics which is bigger than at least two standard deviation from the mean in average task time, total task time, total task GC time, and the number of failed tasks if exists. If there is no outlier, it works like TOTAL_DURATION policy. 3.3.0

Pod template properties

See the below table for the full list of pod specifications that will be overwritten by spark.

Pod Metadata

Pod metadata keyModified valueDescription
name Value of spark.kubernetes.driver.pod.name The driver pod name will be overwritten with either the configured or default value of spark.kubernetes.driver.pod.name. The executor pod names will be unaffected.
namespace Value of spark.kubernetes.namespace Spark makes strong assumptions about the driver and executor namespaces. Both driver and executor namespaces will be replaced by either the configured or default spark conf value.
labels Adds the labels from spark.kubernetes.{driver,executor}.label.* Spark will add additional labels specified by the spark configuration.
annotations Adds the annotations from spark.kubernetes.{driver,executor}.annotation.* Spark will add additional annotations specified by the spark configuration.

Pod Spec

Pod spec keyModified valueDescription
imagePullSecrets Adds image pull secrets from spark.kubernetes.container.image.pullSecrets Additional pull secrets will be added from the spark configuration to both executor pods.
nodeSelector Adds node selectors from spark.kubernetes.node.selector.* Additional node selectors will be added from the spark configuration to both executor pods.
restartPolicy "never" Spark assumes that both drivers and executors never restart.
serviceAccount Value of spark.kubernetes.authenticate.driver.serviceAccountName Spark will override serviceAccount with the value of the spark configuration for only driver pods, and only if the spark configuration is specified. Executor pods will remain unaffected.
serviceAccountName Value of spark.kubernetes.authenticate.driver.serviceAccountName Spark will override serviceAccountName with the value of the spark configuration for only driver pods, and only if the spark configuration is specified. Executor pods will remain unaffected.
volumes Adds volumes from spark.kubernetes.{driver,executor}.volumes.[VolumeType].[VolumeName].mount.path Spark will add volumes as specified by the spark conf, as well as additional volumes necessary for passing spark conf and pod template files.

Container spec

The following affect the driver and executor containers. All other containers in the pod spec will be unaffected.

Container spec keyModified valueDescription
env Adds env variables from spark.kubernetes.driverEnv.[EnvironmentVariableName] Spark will add driver env variables from spark.kubernetes.driverEnv.[EnvironmentVariableName], and executor env variables from spark.executorEnv.[EnvironmentVariableName].
image Value of spark.kubernetes.{driver,executor}.container.image The image will be defined by the spark configurations.
imagePullPolicy Value of spark.kubernetes.container.image.pullPolicy Spark will override the pull policy for both driver and executors.
name See description The container name will be assigned by spark ("spark-kubernetes-driver" for the driver container, and "spark-kubernetes-executor" for each executor container) if not defined by the pod template. If the container is defined by the template, the template's name will be used.
resources See description The cpu limits are set by spark.kubernetes.{driver,executor}.limit.cores. The cpu is set by spark.{driver,executor}.cores. The memory request and limit are set by summing the values of spark.{driver,executor}.memory and spark.{driver,executor}.memoryOverhead. Other resource limits are set by spark.{driver,executor}.resources.{resourceName}.* configs.
volumeMounts Add volumes from spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.{path,readOnly} Spark will add volumes as specified by the spark conf, as well as additional volumes necessary for passing spark conf and pod template files.

Resource Allocation and Configuration Overview

Please make sure to have read the Custom Resource Scheduling and Configuration Overview section on the configuration page. This section only talks about the Kubernetes specific aspects of resource scheduling.

The user is responsible to properly configuring the Kubernetes cluster to have the resources available and ideally isolate each resource per container so that a resource is not shared between multiple containers. If the resource is not isolated the user is responsible for writing a discovery script so that the resource is not shared between containers. See the Kubernetes documentation for specifics on configuring Kubernetes with custom resources.

Spark automatically handles translating the Spark configs spark.{driver/executor}.resource.{resourceType} into the kubernetes configs as long as the Kubernetes resource type follows the Kubernetes device plugin format of vendor-domain/resourcetype. The user must specify the vendor using the spark.{driver/executor}.resource.{resourceType}.vendor config. The user does not need to explicitly add anything if you are using Pod templates. For reference and an example, you can see the Kubernetes documentation for scheduling GPUs. Spark only supports setting the resource limits.

Kubernetes does not tell Spark the addresses of the resources allocated to each container. For that reason, the user must specify a discovery script that gets run by the executor on startup to discover what resources are available to that executor. You can find an example scripts in examples/src/main/scripts/getGpusResources.sh. The script must have execute permissions set and the user should setup permissions to not allow malicious users to modify it. The script should write to STDOUT a JSON string in the format of the ResourceInformation class. This has the resource name and an array of resource addresses available to just that executor.

Resource Level Scheduling Overview

There are several resource level scheduling features supported by Spark on Kubernetes.

Priority Scheduling

Kubernetes supports Pod priority by default.

Spark on Kubernetes allows defining the priority of jobs by Pod template. The user can specify the priorityClassName in driver or executor Pod template spec section. Below is an example to show how to specify it:

apiVersion: v1
Kind: Pod
metadata:
  labels:
    template-label-key: driver-template-label-value
spec:
  # Specify the priority in here 
  priorityClassName: system-node-critical
  containers:
  - name: test-driver-container
    image: will-be-overwritten

Customized Kubernetes Schedulers for Spark on Kubernetes

Spark allows users to specify a custom Kubernetes schedulers.

  1. Specify a scheduler name.

    Users can specify custom scheduler using spark.kubernetes.scheduler.name or spark.kubernetes.{driver/executor}.scheduler.name configuration.

  2. Specify scheduler related configurations.

    To configure the custom scheduler the user can use Pod templates, add labels (spark.kubernetes.{driver,executor}.label.*), annotations (spark.kubernetes.{driver/executor}.annotation.*) or scheduler specific configurations (such as spark.kubernetes.scheduler.volcano.podGroupTemplateFile).

  3. Specify scheduler feature step.

    Users may also consider to use spark.kubernetes.{driver/executor}.pod.featureSteps to support more complex requirements, including but not limited to:

    • Create additional Kubernetes custom resources for driver/executor scheduling.
    • Set scheduler hints according to configuration or existing Pod info dynamically.

Using Volcano as Customized Scheduler for Spark on Kubernetes

This feature is currently experimental. In future versions, there may be behavioral changes around configuration, feature step improvement.

Prerequisites
Build

To create a Spark distribution along with Volcano suppport like those distributed by the Spark Downloads page, also see more in “Building Spark”:

./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phive -Phive-thriftserver -Pkubernetes -Pvolcano
Usage

Spark on Kubernetes allows using Volcano as a custom scheduler. Users can use Volcano to support more advanced resource scheduling: queue scheduling, resource reservation, priority scheduling, and more.

To use Volcano as a custom scheduler the user needs to specify the following configuration options:

# Specify volcano scheduler and PodGroup template
--conf spark.kubernetes.scheduler.name=volcano
--conf spark.kubernetes.scheduler.volcano.podGroupTemplateFile=/path/to/podgroup-template.yaml
# Specify driver/executor VolcanoFeatureStep
--conf spark.kubernetes.driver.pod.featureSteps=org.apache.spark.deploy.k8s.features.VolcanoFeatureStep
--conf spark.kubernetes.executor.pod.featureSteps=org.apache.spark.deploy.k8s.features.VolcanoFeatureStep
Volcano Feature Step

Volcano feature steps help users to create a Volcano PodGroup and set driver/executor pod annotation to link with this PodGroup.

Note that currently only driver/job level PodGroup is supported in Volcano Feature Step.

Volcano PodGroup Template

Volcano defines PodGroup spec using CRD yaml.

Similar to Pod template, Spark users can use Volcano PodGroup Template to define the PodGroup spec configurations. To do so, specify the Spark property spark.kubernetes.scheduler.volcano.podGroupTemplateFile to point to files accessible to the spark-submit process. Below is an example of PodGroup template:

apiVersion: scheduling.volcano.sh/v1beta1
kind: PodGroup
spec:
  # Specify minMember to 1 to make a driver pod
  minMember: 1
  # Specify minResources to support resource reservation (the driver pod resource and executors pod resource should be considered)
  # It is useful for ensource the available resources meet the minimum requirements of the Spark job and avoiding the
  # situation where drivers are scheduled, and then they are unable to schedule sufficient executors to progress.
  minResources:
    cpu: "2"
    memory: "3Gi"
  # Specify the priority, help users to specify job priority in the queue during scheduling.
  priorityClassName: system-node-critical
  # Specify the queue, indicates the resource queue which the job should be submitted to
  queue: default

Using Apache YuniKorn as Customized Scheduler for Spark on Kubernetes

Apache YuniKorn is a resource scheduler for Kubernetes that provides advanced batch scheduling capabilities, such as job queuing, resource fairness, min/max queue capacity and flexible job ordering policies. For available Apache YuniKorn features, please refer to core features.

Prerequisites

Install Apache YuniKorn:

helm repo add yunikorn https://apache.github.io/yunikorn-release
helm repo update
helm install yunikorn yunikorn/yunikorn --namespace yunikorn --version 1.1.0 --create-namespace --set embedAdmissionController=false

The above steps will install YuniKorn v1.1.0 on an existing Kubernetes cluster.

Get started

Submit Spark jobs with the following extra options:

--conf spark.kubernetes.scheduler.name=yunikorn
--conf spark.kubernetes.driver.label.queue=root.default
--conf spark.kubernetes.executor.label.queue=root.default
--conf spark.kubernetes.driver.annotation.yunikorn.apache.org/app-id=
--conf spark.kubernetes.executor.annotation.yunikorn.apache.org/app-id=

Note that `` is the built-in variable that will be substituted with Spark job ID automatically. With the above configuration, the job will be scheduled by YuniKorn scheduler instead of the default Kubernetes scheduler.

Stage Level Scheduling Overview

Stage level scheduling is supported on Kubernetes when dynamic allocation is enabled. This also requires spark.dynamicAllocation.shuffleTracking.enabled to be enabled since Kubernetes doesn’t support an external shuffle service at this time. The order in which containers for different profiles is requested from Kubernetes is not guaranteed. Note that since dynamic allocation on Kubernetes requires the shuffle tracking feature, this means that executors from previous stages that used a different ResourceProfile may not idle timeout due to having shuffle data on them. This could result in using more cluster resources and in the worst case if there are no remaining resources on the Kubernetes cluster then Spark could potentially hang. You may consider looking at config spark.dynamicAllocation.shuffleTracking.timeout to set a timeout, but that could result in data having to be recomputed if the shuffle data is really needed. Note, there is a difference in the way pod template resources are handled between the base default profile and custom ResourceProfiles. Any resources specified in the pod template file will only be used with the base default profile. If you create custom ResourceProfiles be sure to include all necessary resources there since the resources from the template file will not be propagated to custom ResourceProfiles.