Skip to content

Commit

Permalink
[SPARK-22994][K8S] Use a single image for all Spark containers.
Browse files Browse the repository at this point in the history
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 <[email protected]>

Closes apache#20192 from vanzin/SPARK-22994.
  • Loading branch information
Marcelo Vanzin committed Jan 11, 2018
1 parent 6d230dc commit 0b2eefb
Show file tree
Hide file tree
Showing 17 changed files with 189 additions and 226 deletions.
68 changes: 32 additions & 36 deletions sbin/build-push-docker-images.sh → bin/docker-image-tool.sh
Original file line number Diff line number Diff line change
Expand Up @@ -24,29 +24,11 @@ function error {
exit 1
}

# Detect whether this is a git clone or a Spark distribution and adjust paths
# accordingly.
if [ -z "${SPARK_HOME}" ]; then
SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)"
fi
. "${SPARK_HOME}/bin/load-spark-env.sh"

if [ -f "$SPARK_HOME/RELEASE" ]; then
IMG_PATH="kubernetes/dockerfiles"
SPARK_JARS="jars"
else
IMG_PATH="resource-managers/kubernetes/docker/src/main/dockerfiles"
SPARK_JARS="assembly/target/scala-$SPARK_SCALA_VERSION/jars"
fi

if [ ! -d "$IMG_PATH" ]; then
error "Cannot find docker images. This script must be run from a runnable distribution of Apache Spark."
fi

declare -A path=( [spark-driver]="$IMG_PATH/driver/Dockerfile" \
[spark-executor]="$IMG_PATH/executor/Dockerfile" \
[spark-init]="$IMG_PATH/init-container/Dockerfile" )

function image_ref {
local image="$1"
local add_repo="${2:-1}"
Expand All @@ -60,35 +42,49 @@ function image_ref {
}

function build {
docker build \
--build-arg "spark_jars=$SPARK_JARS" \
--build-arg "img_path=$IMG_PATH" \
-t spark-base \
-f "$IMG_PATH/spark-base/Dockerfile" .
for image in "${!path[@]}"; do
docker build -t "$(image_ref $image)" -f ${path[$image]} .
done
local BUILD_ARGS
local IMG_PATH

if [ ! -f "$SPARK_HOME/RELEASE" ]; then
# Set image build arguments accordingly if this is a source repo and not a distribution archive.
IMG_PATH=resource-managers/kubernetes/docker/src/main/dockerfiles
BUILD_ARGS=(
--build-arg
img_path=$IMG_PATH
--build-arg
spark_jars=assembly/target/scala-$SPARK_SCALA_VERSION/jars
)
else
# Not passed as an argument to docker, but used to validate the Spark directory.
IMG_PATH="kubernetes/dockerfiles"
fi

if [ ! -d "$IMG_PATH" ]; then
error "Cannot find docker image. This script must be run from a runnable distribution of Apache Spark."
fi

docker build "${BUILD_ARGS[@]}" \
-t $(image_ref spark) \
-f "$IMG_PATH/spark/Dockerfile" .
}

function push {
for image in "${!path[@]}"; do
docker push "$(image_ref $image)"
done
docker push "$(image_ref spark)"
}

function usage {
cat <<EOF
Usage: $0 [options] [command]
Builds or pushes the built-in Spark Docker images.
Builds or pushes the built-in Spark Docker image.
Commands:
build Build images.
push Push images to a registry. Requires a repository address to be provided, both
when building and when pushing the images.
build Build image. Requires a repository address to be provided if the image will be
pushed to a different registry.
push Push a pre-built image to a registry. Requires a repository address to be provided.
Options:
-r repo Repository address.
-t tag Tag to apply to built images, or to identify images to be pushed.
-t tag Tag to apply to the built image, or to identify the image to be pushed.
-m Use minikube's Docker daemon.
Using minikube when building images will do so directly into minikube's Docker daemon.
Expand All @@ -100,10 +96,10 @@ Check the following documentation for more information on using the minikube Doc
https://kubernetes.io/docs/getting-started-guides/minikube/#reusing-the-docker-daemon
Examples:
- Build images in minikube with tag "testing"
- Build image in minikube with tag "testing"
$0 -m -t testing build
- Build and push images with tag "v2.3.0" to docker.io/myrepo
- Build and push image with tag "v2.3.0" to docker.io/myrepo
$0 -r docker.io/myrepo -t v2.3.0 build
$0 -r docker.io/myrepo -t v2.3.0 push
EOF
Expand Down
58 changes: 26 additions & 32 deletions docs/running-on-kubernetes.md
Original file line number Diff line number Diff line change
Expand Up @@ -53,20 +53,17 @@ in a future release.

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. With Spark 2.3, there are Dockerfiles provided in the runnable distribution that can be customized
and built for your usage.
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.

You may build these docker images from sources.
There is a script, `sbin/build-push-docker-images.sh` that you can use to build and push
customized Spark distribution images consisting of all the above components.
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:

./sbin/build-push-docker-images.sh -r <repo> -t my-tag build
./sbin/build-push-docker-images.sh -r <repo> -t my-tag push

Docker files are under the `kubernetes/dockerfiles/` directory and can be customized further before
building using the supplied script, or manually.
./bin/docker-image-tool.sh -r <repo> -t my-tag build
./bin/docker-image-tool.sh -r <repo> -t my-tag push

## Cluster Mode

Expand All @@ -79,8 +76,7 @@ $ bin/spark-submit \
--name spark-pi \
--class org.apache.spark.examples.SparkPi \
--conf spark.executor.instances=5 \
--conf spark.kubernetes.driver.container.image=<driver-image> \
--conf spark.kubernetes.executor.container.image=<executor-image> \
--conf spark.kubernetes.container.image=<spark-image> \
local:///path/to/examples.jar
```

Expand Down Expand Up @@ -126,13 +122,7 @@ Those dependencies can be added to the classpath by referencing them with `local
### Using Remote Dependencies
When there are application dependencies hosted in remote locations like HDFS or HTTP servers, the driver and executor pods
need a Kubernetes [init-container](https://kubernetes.io/docs/concepts/workloads/pods/init-containers/) for downloading
the dependencies so the driver and executor containers can use them locally. This requires users to specify the container
image for the init-container using the configuration property `spark.kubernetes.initContainer.image`. For example, users
simply add the following option to the `spark-submit` command to specify the init-container image:

```
--conf spark.kubernetes.initContainer.image=<init-container image>
```
the dependencies so the driver and executor containers can use them locally.

The init-container handles remote dependencies specified in `spark.jars` (or the `--jars` option of `spark-submit`) and
`spark.files` (or the `--files` option of `spark-submit`). It also handles remotely hosted main application resources, e.g.,
Expand All @@ -147,9 +137,7 @@ $ bin/spark-submit \
--jars https://path/to/dependency1.jar,https://path/to/dependency2.jar
--files hdfs://host:port/path/to/file1,hdfs://host:port/path/to/file2
--conf spark.executor.instances=5 \
--conf spark.kubernetes.driver.container.image=<driver-image> \
--conf spark.kubernetes.executor.container.image=<executor-image> \
--conf spark.kubernetes.initContainer.image=<init-container image>
--conf spark.kubernetes.container.image=<spark-image> \
https://path/to/examples.jar
```

Expand Down Expand Up @@ -322,21 +310,27 @@ specific to Spark on Kubernetes.
</td>
</tr>
<tr>
<td><code>spark.kubernetes.driver.container.image</code></td>
<td><code>spark.kubernetes.container.image</code></td>
<td><code>(none)</code></td>
<td>
Container image to use for the driver.
This is usually of the form <code>example.com/repo/spark-driver:v1.0.0</code>.
This configuration is required and must be provided by the user.
Container image to use for the Spark application.
This is usually of the form <code>example.com/repo/spark:v1.0.0</code>.
This configuration is required and must be provided by the user, unless explicit
images are provided for each different container type.
</td>
</tr>
<tr>
<td><code>spark.kubernetes.driver.container.image</code></td>
<td><code>(value of spark.kubernetes.container.image)</code></td>
<td>
Custom container image to use for the driver.
</td>
</tr>
<tr>
<td><code>spark.kubernetes.executor.container.image</code></td>
<td><code>(none)</code></td>
<td><code>(value of spark.kubernetes.container.image)</code></td>
<td>
Container image to use for the executors.
This is usually of the form <code>example.com/repo/spark-executor:v1.0.0</code>.
This configuration is required and must be provided by the user.
Custom container image to use for executors.
</td>
</tr>
<tr>
Expand Down Expand Up @@ -643,9 +637,9 @@ specific to Spark on Kubernetes.
</tr>
<tr>
<td><code>spark.kubernetes.initContainer.image</code></td>
<td>(none)</td>
<td><code>(value of spark.kubernetes.container.image)</code></td>
<td>
Container image for the <a href="https://kubernetes.io/docs/concepts/workloads/pods/init-containers/">init-container</a> of the driver and executors for downloading dependencies. This is usually of the form <code>example.com/repo/spark-init:v1.0.0</code>. This configuration is optional and must be provided by the user if any non-container local dependency is used and must be downloaded remotely.
Custom container image for the init container of both driver and executors.
</td>
</tr>
<tr>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,17 +29,23 @@ private[spark] object Config extends Logging {
.stringConf
.createWithDefault("default")

val CONTAINER_IMAGE =
ConfigBuilder("spark.kubernetes.container.image")
.doc("Container image to use for Spark containers. Individual container types " +
"(e.g. driver or executor) can also be configured to use different images if desired, " +
"by setting the container type-specific image name.")
.stringConf
.createOptional

val DRIVER_CONTAINER_IMAGE =
ConfigBuilder("spark.kubernetes.driver.container.image")
.doc("Container image to use for the driver.")
.stringConf
.createOptional
.fallbackConf(CONTAINER_IMAGE)

val EXECUTOR_CONTAINER_IMAGE =
ConfigBuilder("spark.kubernetes.executor.container.image")
.doc("Container image to use for the executors.")
.stringConf
.createOptional
.fallbackConf(CONTAINER_IMAGE)

val CONTAINER_IMAGE_PULL_POLICY =
ConfigBuilder("spark.kubernetes.container.image.pullPolicy")
Expand Down Expand Up @@ -148,8 +154,7 @@ private[spark] object Config extends Logging {
val INIT_CONTAINER_IMAGE =
ConfigBuilder("spark.kubernetes.initContainer.image")
.doc("Image for the driver and executor's init-container for downloading dependencies.")
.stringConf
.createOptional
.fallbackConf(CONTAINER_IMAGE)

val INIT_CONTAINER_MOUNT_TIMEOUT =
ConfigBuilder("spark.kubernetes.mountDependencies.timeout")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,10 +60,9 @@ private[spark] object Constants {
val ENV_APPLICATION_ID = "SPARK_APPLICATION_ID"
val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID"
val ENV_EXECUTOR_POD_IP = "SPARK_EXECUTOR_POD_IP"
val ENV_EXECUTOR_EXTRA_CLASSPATH = "SPARK_EXECUTOR_EXTRA_CLASSPATH"
val ENV_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH"
val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_"
val ENV_SUBMIT_EXTRA_CLASSPATH = "SPARK_SUBMIT_EXTRA_CLASSPATH"
val ENV_CLASSPATH = "SPARK_CLASSPATH"
val ENV_DRIVER_MAIN_CLASS = "SPARK_DRIVER_CLASS"
val ENV_DRIVER_ARGS = "SPARK_DRIVER_ARGS"
val ENV_DRIVER_JAVA_OPTS = "SPARK_DRIVER_JAVA_OPTS"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ private[spark] class InitContainerBootstrap(
.withMountPath(INIT_CONTAINER_PROPERTIES_FILE_DIR)
.endVolumeMount()
.addToVolumeMounts(sharedVolumeMounts: _*)
.addToArgs("init")
.addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH)
.build()

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ private[spark] class BasicDriverConfigurationStep(
override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = {
val driverExtraClasspathEnv = driverExtraClasspath.map { classPath =>
new EnvVarBuilder()
.withName(ENV_SUBMIT_EXTRA_CLASSPATH)
.withName(ENV_CLASSPATH)
.withValue(classPath)
.build()
}
Expand Down Expand Up @@ -133,6 +133,7 @@ private[spark] class BasicDriverConfigurationStep(
.addToLimits("memory", driverMemoryLimitQuantity)
.addToLimits(maybeCpuLimitQuantity.toMap.asJava)
.endResources()
.addToArgs("driver")
.build()

val baseDriverPod = new PodBuilder(driverSpec.driverPod)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ private[spark] class ExecutorPodFactory(
.build()
val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
new EnvVarBuilder()
.withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
.withName(ENV_CLASSPATH)
.withValue(cp)
.build()
}
Expand Down Expand Up @@ -181,6 +181,7 @@ private[spark] class ExecutorPodFactory(
.endResources()
.addAllToEnv(executorEnv.asJava)
.withPorts(requiredPorts.asJava)
.addToArgs("executor")
.build()

val executorPod = new PodBuilder()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,7 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite {
private val SECRET_MOUNT_PATH = "/etc/secrets/driver"

test("Base submission steps with a main app resource.") {
val sparkConf = new SparkConf(false)
.set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE)
val sparkConf = new SparkConf(false).set(CONTAINER_IMAGE, DRIVER_IMAGE)
val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar")
val orchestrator = new DriverConfigOrchestrator(
APP_ID,
Expand All @@ -55,8 +54,7 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite {
}

test("Base submission steps without a main app resource.") {
val sparkConf = new SparkConf(false)
.set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE)
val sparkConf = new SparkConf(false).set(CONTAINER_IMAGE, DRIVER_IMAGE)
val orchestrator = new DriverConfigOrchestrator(
APP_ID,
LAUNCH_TIME,
Expand All @@ -75,8 +73,8 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite {

test("Submission steps with an init-container.") {
val sparkConf = new SparkConf(false)
.set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE)
.set(INIT_CONTAINER_IMAGE, IC_IMAGE)
.set(CONTAINER_IMAGE, DRIVER_IMAGE)
.set(INIT_CONTAINER_IMAGE.key, IC_IMAGE)
.set("spark.jars", "hdfs://localhost:9000/var/apps/jars/jar1.jar")
val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar")
val orchestrator = new DriverConfigOrchestrator(
Expand All @@ -98,7 +96,7 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite {

test("Submission steps with driver secrets to mount") {
val sparkConf = new SparkConf(false)
.set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE)
.set(CONTAINER_IMAGE, DRIVER_IMAGE)
.set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_FOO", SECRET_MOUNT_PATH)
.set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH)
val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ class BasicDriverConfigurationStepSuite extends SparkFunSuite {
.set(KUBERNETES_DRIVER_LIMIT_CORES, "4")
.set(org.apache.spark.internal.config.DRIVER_MEMORY.key, "256M")
.set(org.apache.spark.internal.config.DRIVER_MEMORY_OVERHEAD, 200L)
.set(DRIVER_CONTAINER_IMAGE, "spark-driver:latest")
.set(CONTAINER_IMAGE, "spark-driver:latest")
.set(s"$KUBERNETES_DRIVER_ANNOTATION_PREFIX$CUSTOM_ANNOTATION_KEY", CUSTOM_ANNOTATION_VALUE)
.set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY1", "customDriverEnv1")
.set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY2", "customDriverEnv2")
Expand Down Expand Up @@ -79,7 +79,7 @@ class BasicDriverConfigurationStepSuite extends SparkFunSuite {
.asScala
.map(env => (env.getName, env.getValue))
.toMap
assert(envs(ENV_SUBMIT_EXTRA_CLASSPATH) === "/opt/spark/spark-examples.jar")
assert(envs(ENV_CLASSPATH) === "/opt/spark/spark-examples.jar")
assert(envs(ENV_DRIVER_MEMORY) === "256M")
assert(envs(ENV_DRIVER_MAIN_CLASS) === MAIN_CLASS)
assert(envs(ENV_DRIVER_ARGS) === "arg1 arg2 \"arg 3\"")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ class InitContainerConfigOrchestratorSuite extends SparkFunSuite {

test("including basic configuration step") {
val sparkConf = new SparkConf(true)
.set(INIT_CONTAINER_IMAGE, DOCKER_IMAGE)
.set(CONTAINER_IMAGE, DOCKER_IMAGE)
.set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE)

val orchestrator = new InitContainerConfigOrchestrator(
Expand All @@ -59,7 +59,7 @@ class InitContainerConfigOrchestratorSuite extends SparkFunSuite {

test("including step to mount user-specified secrets") {
val sparkConf = new SparkConf(false)
.set(INIT_CONTAINER_IMAGE, DOCKER_IMAGE)
.set(CONTAINER_IMAGE, DOCKER_IMAGE)
.set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_FOO", SECRET_MOUNT_PATH)
.set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH)

Expand Down
Loading

0 comments on commit 0b2eefb

Please sign in to comment.