Skip to content

Commit 3f4060c

Browse files
liyinan926Marcelo Vanzin
authored and
Marcelo Vanzin
committed
[SPARK-22646][K8S] Spark on Kubernetes - basic submission client
This PR contains implementation of the basic submission client for the cluster mode of Spark on Kubernetes. It's step 2 from the step-wise plan documented [here](apache-spark-on-k8s#441 (comment)). This addition is covered by the [SPIP](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) vote which passed on Aug 31. This PR and #19468 together form a MVP of Spark on Kubernetes that allows users to run Spark applications that use resources locally within the driver and executor containers on Kubernetes 1.6 and up. Some changes on pom and build/test setup are copied over from #19468 to make this PR self contained and testable. The submission client is mainly responsible for creating the Kubernetes pod that runs the Spark driver. It follows a step-based approach to construct the driver pod, as the code under the `submit.steps` package shows. The steps are orchestrated by `DriverConfigurationStepsOrchestrator`. `Client` creates the driver pod and waits for the application to complete if it's configured to do so, which is the case by default. This PR also contains Dockerfiles of the driver and executor images. They are included because some of the environment variables set in the code would not make sense without referring to the Dockerfiles. * The patch contains unit tests which are passing. * Manual testing: ./build/mvn -Pkubernetes clean package succeeded. * It is a subset of the entire changelist hosted at http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations. * There is integration testing enabled in the fork currently hosted by PepperData which is being moved over to RiseLAB CI. * Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html cc rxin felixcheung mateiz (shepherd) k8s-big-data SIG members & contributors: mccheah foxish ash211 ssuchter varunkatta kimoonkim erikerlandson tnachen ifilonenko liyinan926 Author: Yinan Li <[email protected]> Closes #19717 from liyinan926/spark-kubernetes-4.
1 parent c235b5f commit 3f4060c

39 files changed

+2566
-67
lines changed

assembly/pom.xml

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -148,6 +148,16 @@
148148
</dependency>
149149
</dependencies>
150150
</profile>
151+
<profile>
152+
<id>kubernetes</id>
153+
<dependencies>
154+
<dependency>
155+
<groupId>org.apache.spark</groupId>
156+
<artifactId>spark-kubernetes_${scala.binary.version}</artifactId>
157+
<version>${project.version}</version>
158+
</dependency>
159+
</dependencies>
160+
</profile>
151161
<profile>
152162
<id>hive</id>
153163
<dependencies>

core/src/main/scala/org/apache/spark/SparkConf.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -668,7 +668,11 @@ private[spark] object SparkConf extends Logging {
668668
MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM.key -> Seq(
669669
AlternateConfig("spark.reducer.maxReqSizeShuffleToMem", "2.3")),
670670
LISTENER_BUS_EVENT_QUEUE_CAPACITY.key -> Seq(
671-
AlternateConfig("spark.scheduler.listenerbus.eventqueue.size", "2.3"))
671+
AlternateConfig("spark.scheduler.listenerbus.eventqueue.size", "2.3")),
672+
DRIVER_MEMORY_OVERHEAD.key -> Seq(
673+
AlternateConfig("spark.yarn.driver.memoryOverhead", "2.3")),
674+
EXECUTOR_MEMORY_OVERHEAD.key -> Seq(
675+
AlternateConfig("spark.yarn.executor.memoryOverhead", "2.3"))
672676
)
673677

674678
/**

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package org.apache.spark
1919

2020
import java.io._
21-
import java.lang.reflect.Constructor
2221
import java.net.URI
2322
import java.util.{Arrays, Locale, Properties, ServiceLoader, UUID}
2423
import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}

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

Lines changed: 41 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -76,7 +76,8 @@ object SparkSubmit extends CommandLineUtils with Logging {
7676
private val STANDALONE = 2
7777
private val MESOS = 4
7878
private val LOCAL = 8
79-
private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL
79+
private val KUBERNETES = 16
80+
private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL | KUBERNETES
8081

8182
// Deploy modes
8283
private val CLIENT = 1
@@ -97,6 +98,8 @@ object SparkSubmit extends CommandLineUtils with Logging {
9798
"org.apache.spark.deploy.yarn.YarnClusterApplication"
9899
private[deploy] val REST_CLUSTER_SUBMIT_CLASS = classOf[RestSubmissionClientApp].getName()
99100
private[deploy] val STANDALONE_CLUSTER_SUBMIT_CLASS = classOf[ClientApp].getName()
101+
private[deploy] val KUBERNETES_CLUSTER_SUBMIT_CLASS =
102+
"org.apache.spark.deploy.k8s.submit.KubernetesClientApplication"
100103

101104
// scalastyle:off println
102105
private[spark] def printVersionAndExit(): Unit = {
@@ -257,9 +260,10 @@ object SparkSubmit extends CommandLineUtils with Logging {
257260
YARN
258261
case m if m.startsWith("spark") => STANDALONE
259262
case m if m.startsWith("mesos") => MESOS
263+
case m if m.startsWith("k8s") => KUBERNETES
260264
case m if m.startsWith("local") => LOCAL
261265
case _ =>
262-
printErrorAndExit("Master must either be yarn or start with spark, mesos, local")
266+
printErrorAndExit("Master must either be yarn or start with spark, mesos, k8s, or local")
263267
-1
264268
}
265269

@@ -294,6 +298,16 @@ object SparkSubmit extends CommandLineUtils with Logging {
294298
}
295299
}
296300

301+
if (clusterManager == KUBERNETES) {
302+
args.master = Utils.checkAndGetK8sMasterUrl(args.master)
303+
// Make sure KUBERNETES is included in our build if we're trying to use it
304+
if (!Utils.classIsLoadable(KUBERNETES_CLUSTER_SUBMIT_CLASS) && !Utils.isTesting) {
305+
printErrorAndExit(
306+
"Could not load KUBERNETES classes. " +
307+
"This copy of Spark may not have been compiled with KUBERNETES support.")
308+
}
309+
}
310+
297311
// Fail fast, the following modes are not supported or applicable
298312
(clusterManager, deployMode) match {
299313
case (STANDALONE, CLUSTER) if args.isPython =>
@@ -302,6 +316,12 @@ object SparkSubmit extends CommandLineUtils with Logging {
302316
case (STANDALONE, CLUSTER) if args.isR =>
303317
printErrorAndExit("Cluster deploy mode is currently not supported for R " +
304318
"applications on standalone clusters.")
319+
case (KUBERNETES, _) if args.isPython =>
320+
printErrorAndExit("Python applications are currently not supported for Kubernetes.")
321+
case (KUBERNETES, _) if args.isR =>
322+
printErrorAndExit("R applications are currently not supported for Kubernetes.")
323+
case (KUBERNETES, CLIENT) =>
324+
printErrorAndExit("Client mode is currently not supported for Kubernetes.")
305325
case (LOCAL, CLUSTER) =>
306326
printErrorAndExit("Cluster deploy mode is not compatible with master \"local\"")
307327
case (_, CLUSTER) if isShell(args.primaryResource) =>
@@ -322,6 +342,7 @@ object SparkSubmit extends CommandLineUtils with Logging {
322342
val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER
323343
val isMesosCluster = clusterManager == MESOS && deployMode == CLUSTER
324344
val isStandAloneCluster = clusterManager == STANDALONE && deployMode == CLUSTER
345+
val isKubernetesCluster = clusterManager == KUBERNETES && deployMode == CLUSTER
325346

326347
if (!isMesosCluster && !isStandAloneCluster) {
327348
// Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files
@@ -557,19 +578,19 @@ object SparkSubmit extends CommandLineUtils with Logging {
557578
OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.keytab"),
558579

559580
// Other options
560-
OptionAssigner(args.executorCores, STANDALONE | YARN, ALL_DEPLOY_MODES,
581+
OptionAssigner(args.executorCores, STANDALONE | YARN | KUBERNETES, ALL_DEPLOY_MODES,
561582
confKey = "spark.executor.cores"),
562-
OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES,
583+
OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN | KUBERNETES, ALL_DEPLOY_MODES,
563584
confKey = "spark.executor.memory"),
564-
OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES,
585+
OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES,
565586
confKey = "spark.cores.max"),
566587
OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES,
567588
confKey = "spark.files"),
568589
OptionAssigner(args.jars, LOCAL, CLIENT, confKey = "spark.jars"),
569590
OptionAssigner(args.jars, STANDALONE | MESOS, ALL_DEPLOY_MODES, confKey = "spark.jars"),
570-
OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN, CLUSTER,
591+
OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN | KUBERNETES, CLUSTER,
571592
confKey = "spark.driver.memory"),
572-
OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN, CLUSTER,
593+
OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN | KUBERNETES, CLUSTER,
573594
confKey = "spark.driver.cores"),
574595
OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER,
575596
confKey = "spark.driver.supervise"),
@@ -703,6 +724,19 @@ object SparkSubmit extends CommandLineUtils with Logging {
703724
}
704725
}
705726

727+
if (isKubernetesCluster) {
728+
childMainClass = KUBERNETES_CLUSTER_SUBMIT_CLASS
729+
if (args.primaryResource != SparkLauncher.NO_RESOURCE) {
730+
childArgs ++= Array("--primary-java-resource", args.primaryResource)
731+
}
732+
childArgs ++= Array("--main-class", args.mainClass)
733+
if (args.childArgs != null) {
734+
args.childArgs.foreach { arg =>
735+
childArgs += ("--arg", arg)
736+
}
737+
}
738+
}
739+
706740
// Load any properties specified through --conf and the default properties file
707741
for ((k, v) <- args.sparkProperties) {
708742
sparkConf.setIfMissing(k, v)

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -515,8 +515,8 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
515515
outStream.println(
516516
s"""
517517
|Options:
518-
| --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local
519-
| (Default: local[*]).
518+
| --master MASTER_URL spark://host:port, mesos://host:port, yarn,
519+
| k8s://https://host:port, or local (Default: local[*]).
520520
| --deploy-mode DEPLOY_MODE Whether to launch the driver program locally ("client") or
521521
| on one of the worker machines inside the cluster ("cluster")
522522
| (Default: client).

core/src/main/scala/org/apache/spark/internal/config/package.scala

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,10 @@ package object config {
4141
.bytesConf(ByteUnit.MiB)
4242
.createWithDefaultString("1g")
4343

44+
private[spark] val DRIVER_MEMORY_OVERHEAD = ConfigBuilder("spark.driver.memoryOverhead")
45+
.bytesConf(ByteUnit.MiB)
46+
.createOptional
47+
4448
private[spark] val EVENT_LOG_COMPRESS =
4549
ConfigBuilder("spark.eventLog.compress")
4650
.booleanConf
@@ -80,6 +84,10 @@ package object config {
8084
.bytesConf(ByteUnit.MiB)
8185
.createWithDefaultString("1g")
8286

87+
private[spark] val EXECUTOR_MEMORY_OVERHEAD = ConfigBuilder("spark.executor.memoryOverhead")
88+
.bytesConf(ByteUnit.MiB)
89+
.createOptional
90+
8391
private[spark] val MEMORY_OFFHEAP_ENABLED = ConfigBuilder("spark.memory.offHeap.enabled")
8492
.doc("If true, Spark will attempt to use off-heap memory for certain operations. " +
8593
"If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive.")

core/src/main/scala/org/apache/spark/util/Utils.scala

Lines changed: 36 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2744,6 +2744,42 @@ private[spark] object Utils extends Logging {
27442744
}
27452745
}
27462746

2747+
/**
2748+
* Check the validity of the given Kubernetes master URL and return the resolved URL. Prefix
2749+
* "k8s:" is appended to the resolved URL as the prefix is used by KubernetesClusterManager
2750+
* in canCreate to determine if the KubernetesClusterManager should be used.
2751+
*/
2752+
def checkAndGetK8sMasterUrl(rawMasterURL: String): String = {
2753+
require(rawMasterURL.startsWith("k8s://"),
2754+
"Kubernetes master URL must start with k8s://.")
2755+
val masterWithoutK8sPrefix = rawMasterURL.substring("k8s://".length)
2756+
2757+
// To handle master URLs, e.g., k8s://host:port.
2758+
if (!masterWithoutK8sPrefix.contains("://")) {
2759+
val resolvedURL = s"https://$masterWithoutK8sPrefix"
2760+
logInfo("No scheme specified for kubernetes master URL, so defaulting to https. Resolved " +
2761+
s"URL is $resolvedURL.")
2762+
return s"k8s:$resolvedURL"
2763+
}
2764+
2765+
val masterScheme = new URI(masterWithoutK8sPrefix).getScheme
2766+
val resolvedURL = masterScheme.toLowerCase match {
2767+
case "https" =>
2768+
masterWithoutK8sPrefix
2769+
case "http" =>
2770+
logWarning("Kubernetes master URL uses HTTP instead of HTTPS.")
2771+
masterWithoutK8sPrefix
2772+
case null =>
2773+
val resolvedURL = s"https://$masterWithoutK8sPrefix"
2774+
logInfo("No scheme specified for kubernetes master URL, so defaulting to https. Resolved " +
2775+
s"URL is $resolvedURL.")
2776+
resolvedURL
2777+
case _ =>
2778+
throw new IllegalArgumentException("Invalid Kubernetes master scheme: " + masterScheme)
2779+
}
2780+
2781+
return s"k8s:$resolvedURL"
2782+
}
27472783
}
27482784

27492785
private[util] object CallerContext extends Logging {

core/src/test/scala/org/apache/spark/SparkContextSuite.scala

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -550,6 +550,12 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
550550
}
551551
}
552552

553+
test("client mode with a k8s master url") {
554+
intercept[SparkException] {
555+
sc = new SparkContext("k8s://https://host:port", "test", new SparkConf())
556+
}
557+
}
558+
553559
testCancellingTasks("that raise interrupted exception on cancel") {
554560
Thread.sleep(9999999)
555561
}

core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -388,6 +388,33 @@ class SparkSubmitSuite
388388
conf.get("spark.ui.enabled") should be ("false")
389389
}
390390

391+
test("handles k8s cluster mode") {
392+
val clArgs = Seq(
393+
"--deploy-mode", "cluster",
394+
"--master", "k8s://host:port",
395+
"--executor-memory", "5g",
396+
"--class", "org.SomeClass",
397+
"--driver-memory", "4g",
398+
"--conf", "spark.kubernetes.namespace=spark",
399+
"--conf", "spark.kubernetes.driver.docker.image=bar",
400+
"/home/thejar.jar",
401+
"arg1")
402+
val appArgs = new SparkSubmitArguments(clArgs)
403+
val (childArgs, classpath, conf, mainClass) = prepareSubmitEnvironment(appArgs)
404+
405+
val childArgsMap = childArgs.grouped(2).map(a => a(0) -> a(1)).toMap
406+
childArgsMap.get("--primary-java-resource") should be (Some("file:/home/thejar.jar"))
407+
childArgsMap.get("--main-class") should be (Some("org.SomeClass"))
408+
childArgsMap.get("--arg") should be (Some("arg1"))
409+
mainClass should be (KUBERNETES_CLUSTER_SUBMIT_CLASS)
410+
classpath should have length (0)
411+
conf.get("spark.master") should be ("k8s:https://host:port")
412+
conf.get("spark.executor.memory") should be ("5g")
413+
conf.get("spark.driver.memory") should be ("4g")
414+
conf.get("spark.kubernetes.namespace") should be ("spark")
415+
conf.get("spark.kubernetes.driver.docker.image") should be ("bar")
416+
}
417+
391418
test("handles confs with flag equivalents") {
392419
val clArgs = Seq(
393420
"--deploy-mode", "cluster",

core/src/test/scala/org/apache/spark/util/UtilsSuite.scala

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1146,6 +1146,27 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
11461146
}
11471147
}
11481148

1149+
test("check Kubernetes master URL") {
1150+
val k8sMasterURLHttps = Utils.checkAndGetK8sMasterUrl("k8s://https://host:port")
1151+
assert(k8sMasterURLHttps === "k8s:https://host:port")
1152+
1153+
val k8sMasterURLHttp = Utils.checkAndGetK8sMasterUrl("k8s://http://host:port")
1154+
assert(k8sMasterURLHttp === "k8s:http://host:port")
1155+
1156+
val k8sMasterURLWithoutScheme = Utils.checkAndGetK8sMasterUrl("k8s://127.0.0.1:8443")
1157+
assert(k8sMasterURLWithoutScheme === "k8s:https://127.0.0.1:8443")
1158+
1159+
val k8sMasterURLWithoutScheme2 = Utils.checkAndGetK8sMasterUrl("k8s://127.0.0.1")
1160+
assert(k8sMasterURLWithoutScheme2 === "k8s:https://127.0.0.1")
1161+
1162+
intercept[IllegalArgumentException] {
1163+
Utils.checkAndGetK8sMasterUrl("k8s:https://host:port")
1164+
}
1165+
1166+
intercept[IllegalArgumentException] {
1167+
Utils.checkAndGetK8sMasterUrl("k8s://foo://host:port")
1168+
}
1169+
}
11491170
}
11501171

11511172
private class SimpleExtension

docs/configuration.md

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -157,13 +157,33 @@ of the most common options to set are:
157157
or in your default properties file.
158158
</td>
159159
</tr>
160+
<tr>
161+
<td><code>spark.driver.memoryOverhead</code></td>
162+
<td>driverMemory * 0.10, with minimum of 384 </td>
163+
<td>
164+
The amount of off-heap memory (in megabytes) to be allocated per driver in cluster mode. This is
165+
memory that accounts for things like VM overheads, interned strings, other native overheads, etc.
166+
This tends to grow with the container size (typically 6-10%). This option is currently supported
167+
on YARN and Kubernetes.
168+
</td>
169+
</tr>
160170
<tr>
161171
<td><code>spark.executor.memory</code></td>
162172
<td>1g</td>
163173
<td>
164174
Amount of memory to use per executor process (e.g. <code>2g</code>, <code>8g</code>).
165175
</td>
166176
</tr>
177+
<tr>
178+
<td><code>spark.executor.memoryOverhead</code></td>
179+
<td>executorMemory * 0.10, with minimum of 384 </td>
180+
<td>
181+
The amount of off-heap memory (in megabytes) to be allocated per executor. This is memory that
182+
accounts for things like VM overheads, interned strings, other native overheads, etc. This tends
183+
to grow with the executor size (typically 6-10%). This option is currently supported on YARN and
184+
Kubernetes.
185+
</td>
186+
</tr>
167187
<tr>
168188
<td><code>spark.extraListeners</code></td>
169189
<td>(none)</td>

docs/running-on-yarn.md

Lines changed: 1 addition & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -227,25 +227,11 @@ To use a custom metrics.properties for the application master and executors, upd
227227
The number of executors for static allocation. With <code>spark.dynamicAllocation.enabled</code>, the initial set of executors will be at least this large.
228228
</td>
229229
</tr>
230-
<tr>
231-
<td><code>spark.yarn.executor.memoryOverhead</code></td>
232-
<td>executorMemory * 0.10, with minimum of 384 </td>
233-
<td>
234-
The amount of off-heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the executor size (typically 6-10%).
235-
</td>
236-
</tr>
237-
<tr>
238-
<td><code>spark.yarn.driver.memoryOverhead</code></td>
239-
<td>driverMemory * 0.10, with minimum of 384 </td>
240-
<td>
241-
The amount of off-heap memory (in megabytes) to be allocated per driver in cluster mode. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the container size (typically 6-10%).
242-
</td>
243-
</tr>
244230
<tr>
245231
<td><code>spark.yarn.am.memoryOverhead</code></td>
246232
<td>AM memory * 0.10, with minimum of 384 </td>
247233
<td>
248-
Same as <code>spark.yarn.driver.memoryOverhead</code>, but for the YARN Application Master in client mode.
234+
Same as <code>spark.driver.memoryOverhead</code>, but for the YARN Application Master in client mode.
249235
</td>
250236
</tr>
251237
<tr>

launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -82,7 +82,7 @@ class SparkSubmitOptionParser {
8282
* name of the option, passed to {@link #handle(String, String)}.
8383
* <p>
8484
* Options not listed here nor in the "switch" list below will result in a call to
85-
* {@link $#handleUnknown(String)}.
85+
* {@link #handleUnknown(String)}.
8686
* <p>
8787
* These two arrays are visible for tests.
8888
*/

0 commit comments

Comments
 (0)