Skip to content

Commit 6d23bae

Browse files
ifilonenkomccheah
authored andcommitted
[SPARK-22839][K8S] Remove the use of init-container for downloading remote dependencies
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. This patch was tested with unit and integration tests. Author: Ilan Filonenko <[email protected]> Closes apache#20669 from ifilonenko/remove-init-container.
1 parent 4ac2432 commit 6d23bae

35 files changed

+544
-2096
lines changed

bin/docker-image-tool.sh

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -63,9 +63,11 @@ function build {
6363
error "Cannot find docker image. This script must be run from a runnable distribution of Apache Spark."
6464
fi
6565

66+
local DOCKERFILE=${DOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
67+
6668
docker build "${BUILD_ARGS[@]}" \
6769
-t $(image_ref spark) \
68-
-f "$IMG_PATH/spark/Dockerfile" .
70+
-f "$DOCKERFILE" .
6971
}
7072

7173
function push {
@@ -83,6 +85,7 @@ Commands:
8385
push Push a pre-built image to a registry. Requires a repository address to be provided.
8486
8587
Options:
88+
-f file Dockerfile to build. By default builds the Dockerfile shipped with Spark.
8689
-r repo Repository address.
8790
-t tag Tag to apply to the built image, or to identify the image to be pushed.
8891
-m Use minikube's Docker daemon.
@@ -112,10 +115,12 @@ fi
112115

113116
REPO=
114117
TAG=
115-
while getopts mr:t: option
118+
DOCKERFILE=
119+
while getopts f:mr:t: option
116120
do
117121
case "${option}"
118122
in
123+
f) DOCKERFILE=${OPTARG};;
119124
r) REPO=${OPTARG};;
120125
t) TAG=${OPTARG};;
121126
m)

core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -320,8 +320,6 @@ object SparkSubmit extends CommandLineUtils with Logging {
320320
printErrorAndExit("Python applications are currently not supported for Kubernetes.")
321321
case (KUBERNETES, _) if args.isR =>
322322
printErrorAndExit("R applications are currently not supported for Kubernetes.")
323-
case (KUBERNETES, CLIENT) =>
324-
printErrorAndExit("Client mode is currently not supported for Kubernetes.")
325323
case (LOCAL, CLUSTER) =>
326324
printErrorAndExit("Cluster deploy mode is not compatible with master \"local\"")
327325
case (_, CLUSTER) if isShell(args.primaryResource) =>

docs/running-on-kubernetes.md

Lines changed: 305 additions & 500 deletions
Large diffs are not rendered by default.
Lines changed: 48 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,48 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
// scalastyle:off println
19+
package org.apache.spark.examples
20+
21+
import java.io.File
22+
23+
import org.apache.spark.SparkFiles
24+
import org.apache.spark.sql.SparkSession
25+
26+
/** Usage: SparkRemoteFileTest [file] */
27+
object SparkRemoteFileTest {
28+
def main(args: Array[String]) {
29+
if (args.length < 1) {
30+
System.err.println("Usage: SparkRemoteFileTest <file>")
31+
System.exit(1)
32+
}
33+
val spark = SparkSession
34+
.builder()
35+
.appName("SparkRemoteFileTest")
36+
.getOrCreate()
37+
val sc = spark.sparkContext
38+
val rdd = sc.parallelize(Seq(1)).map(_ => {
39+
val localLocation = SparkFiles.get(args(0))
40+
println(s"${args(0)} is stored at: $localLocation")
41+
new File(localLocation).isFile
42+
})
43+
val truthCheck = rdd.collect().head
44+
println(s"Mounting of ${args(0)} was $truthCheck")
45+
spark.stop()
46+
}
47+
}
48+
// scalastyle:on println

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala

Lines changed: 6 additions & 67 deletions
Original file line numberDiff line numberDiff line change
@@ -79,6 +79,12 @@ private[spark] object Config extends Logging {
7979
.stringConf
8080
.createOptional
8181

82+
val KUBERNETES_DRIVER_SUBMIT_CHECK =
83+
ConfigBuilder("spark.kubernetes.submitInDriver")
84+
.internal()
85+
.booleanConf
86+
.createOptional
87+
8288
val KUBERNETES_EXECUTOR_LIMIT_CORES =
8389
ConfigBuilder("spark.kubernetes.executor.limit.cores")
8490
.doc("Specify the hard cpu limit for each executor pod")
@@ -135,73 +141,6 @@ private[spark] object Config extends Logging {
135141
.checkValue(interval => interval > 0, s"Logging interval must be a positive time value.")
136142
.createWithDefaultString("1s")
137143

138-
val JARS_DOWNLOAD_LOCATION =
139-
ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir")
140-
.doc("Location to download jars to in the driver and executors. When using " +
141-
"spark-submit, this directory must be empty and will be mounted as an empty directory " +
142-
"volume on the driver and executor pod.")
143-
.stringConf
144-
.createWithDefault("/var/spark-data/spark-jars")
145-
146-
val FILES_DOWNLOAD_LOCATION =
147-
ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir")
148-
.doc("Location to download files to in the driver and executors. When using " +
149-
"spark-submit, this directory must be empty and will be mounted as an empty directory " +
150-
"volume on the driver and executor pods.")
151-
.stringConf
152-
.createWithDefault("/var/spark-data/spark-files")
153-
154-
val INIT_CONTAINER_IMAGE =
155-
ConfigBuilder("spark.kubernetes.initContainer.image")
156-
.doc("Image for the driver and executor's init-container for downloading dependencies.")
157-
.fallbackConf(CONTAINER_IMAGE)
158-
159-
val INIT_CONTAINER_MOUNT_TIMEOUT =
160-
ConfigBuilder("spark.kubernetes.mountDependencies.timeout")
161-
.doc("Timeout before aborting the attempt to download and unpack dependencies from remote " +
162-
"locations into the driver and executor pods.")
163-
.timeConf(TimeUnit.SECONDS)
164-
.createWithDefault(300)
165-
166-
val INIT_CONTAINER_MAX_THREAD_POOL_SIZE =
167-
ConfigBuilder("spark.kubernetes.mountDependencies.maxSimultaneousDownloads")
168-
.doc("Maximum number of remote dependencies to download simultaneously in a driver or " +
169-
"executor pod.")
170-
.intConf
171-
.createWithDefault(5)
172-
173-
val INIT_CONTAINER_REMOTE_JARS =
174-
ConfigBuilder("spark.kubernetes.initContainer.remoteJars")
175-
.doc("Comma-separated list of jar URIs to download in the init-container. This is " +
176-
"calculated from spark.jars.")
177-
.internal()
178-
.stringConf
179-
.createOptional
180-
181-
val INIT_CONTAINER_REMOTE_FILES =
182-
ConfigBuilder("spark.kubernetes.initContainer.remoteFiles")
183-
.doc("Comma-separated list of file URIs to download in the init-container. This is " +
184-
"calculated from spark.files.")
185-
.internal()
186-
.stringConf
187-
.createOptional
188-
189-
val INIT_CONTAINER_CONFIG_MAP_NAME =
190-
ConfigBuilder("spark.kubernetes.initContainer.configMapName")
191-
.doc("Name of the config map to use in the init-container that retrieves submitted files " +
192-
"for the executor.")
193-
.internal()
194-
.stringConf
195-
.createOptional
196-
197-
val INIT_CONTAINER_CONFIG_MAP_KEY_CONF =
198-
ConfigBuilder("spark.kubernetes.initContainer.configMapKey")
199-
.doc("Key for the entry in the init container config map for submitted files that " +
200-
"corresponds to the properties for this init-container.")
201-
.internal()
202-
.stringConf
203-
.createOptional
204-
205144
val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX =
206145
"spark.kubernetes.authenticate.submission"
207146

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala

Lines changed: 6 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -63,22 +63,13 @@ private[spark] object Constants {
6363
val ENV_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH"
6464
val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_"
6565
val ENV_CLASSPATH = "SPARK_CLASSPATH"
66-
val ENV_DRIVER_MAIN_CLASS = "SPARK_DRIVER_CLASS"
67-
val ENV_DRIVER_ARGS = "SPARK_DRIVER_ARGS"
68-
val ENV_DRIVER_JAVA_OPTS = "SPARK_DRIVER_JAVA_OPTS"
6966
val ENV_DRIVER_BIND_ADDRESS = "SPARK_DRIVER_BIND_ADDRESS"
70-
val ENV_DRIVER_MEMORY = "SPARK_DRIVER_MEMORY"
71-
val ENV_MOUNTED_FILES_DIR = "SPARK_MOUNTED_FILES_DIR"
72-
73-
// Bootstrapping dependencies with the init-container
74-
val INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME = "download-jars-volume"
75-
val INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME = "download-files-volume"
76-
val INIT_CONTAINER_PROPERTIES_FILE_VOLUME = "spark-init-properties"
77-
val INIT_CONTAINER_PROPERTIES_FILE_DIR = "/etc/spark-init"
78-
val INIT_CONTAINER_PROPERTIES_FILE_NAME = "spark-init.properties"
79-
val INIT_CONTAINER_PROPERTIES_FILE_PATH =
80-
s"$INIT_CONTAINER_PROPERTIES_FILE_DIR/$INIT_CONTAINER_PROPERTIES_FILE_NAME"
81-
val INIT_CONTAINER_SECRET_VOLUME_NAME = "spark-init-secret"
67+
val ENV_SPARK_CONF_DIR = "SPARK_CONF_DIR"
68+
// Spark app configs for containers
69+
val SPARK_CONF_VOLUME = "spark-conf-volume"
70+
val SPARK_CONF_DIR_INTERNAL = "/opt/spark/conf"
71+
val SPARK_CONF_FILE_NAME = "spark.properties"
72+
val SPARK_CONF_PATH = s"$SPARK_CONF_DIR_INTERNAL/$SPARK_CONF_FILE_NAME"
8273

8374
// Miscellaneous
8475
val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc"

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala

Lines changed: 0 additions & 120 deletions
This file was deleted.

0 commit comments

Comments
 (0)