Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.

Commit 20956e7

Browse files
committed
Dynamic allocation (#272)
* dynamic allocation: shuffle service docker, yaml and test fixture * dynamic allocation: changes to spark-core * dynamic allocation: tests * dynamic allocation: docs * dynamic allocation: kubernetes allocator and executor accounting * dynamic allocation: shuffle service, node caching
1 parent 2af7f05 commit 20956e7

File tree

18 files changed

+683
-60
lines changed

18 files changed

+683
-60
lines changed

conf/kubernetes-shuffle-service.yaml

Lines changed: 53 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,53 @@
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+
apiVersion: extensions/v1beta1
19+
kind: DaemonSet
20+
metadata:
21+
labels:
22+
app: spark-shuffle-service
23+
spark-version: 2.1.0
24+
name: shuffle
25+
spec:
26+
template:
27+
metadata:
28+
labels:
29+
app: spark-shuffle-service
30+
spark-version: 2.1.0
31+
spec:
32+
volumes:
33+
- name: temp-volume
34+
hostPath:
35+
path: '/var/tmp' # change this path according to your cluster configuration.
36+
containers:
37+
- name: shuffle
38+
# This is an official image that is built
39+
# from the dockerfiles/shuffle directory
40+
# in the spark distribution.
41+
image: kubespark/spark-shuffle:v2.1.0-kubernetes-0.1.0-alpha.3
42+
volumeMounts:
43+
- mountPath: '/tmp'
44+
name: temp-volume
45+
# more volumes can be mounted here.
46+
# The spark job must be configured to use these
47+
# mounts using the configuration:
48+
# spark.kubernetes.shuffle.dir=<mount-1>,<mount-2>,...
49+
resources:
50+
requests:
51+
cpu: "1"
52+
limits:
53+
cpu: "1"

core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -200,7 +200,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
200200
new SecurityManager(executorConf),
201201
clientMode = true)
202202
val driver = fetcher.setupEndpointRefByURI(driverUrl)
203-
val cfg = driver.askSync[SparkAppConfig](RetrieveSparkAppConfig)
203+
val cfg = driver.askSync[SparkAppConfig](RetrieveSparkAppConfig(executorId))
204204
val props = cfg.sparkProperties ++ Seq[(String, String)](("spark.app.id", appId))
205205
fetcher.shutdown()
206206

core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@ private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable
2828

2929
private[spark] object CoarseGrainedClusterMessages {
3030

31-
case object RetrieveSparkAppConfig extends CoarseGrainedClusterMessage
31+
case class RetrieveSparkAppConfig(executorId: String) extends CoarseGrainedClusterMessage
3232

3333
case class SparkAppConfig(
3434
sparkProperties: Seq[(String, String)],

core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -219,7 +219,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
219219
removeExecutor(executorId, reason)
220220
context.reply(true)
221221

222-
case RetrieveSparkAppConfig =>
222+
case RetrieveSparkAppConfig(executorId) =>
223223
val reply = SparkAppConfig(sparkProperties,
224224
SparkEnv.get.securityManager.getIOEncryptionKey())
225225
context.reply(reply)

core/src/main/scala/org/apache/spark/storage/BlockManager.scala

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -233,8 +233,14 @@ private[spark] class BlockManager(
233233
blockManagerId = if (idFromMaster != null) idFromMaster else id
234234

235235
shuffleServerId = if (externalShuffleServiceEnabled) {
236-
logInfo(s"external shuffle service port = $externalShuffleServicePort")
237-
BlockManagerId(executorId, blockTransferService.hostName, externalShuffleServicePort)
236+
val shuffleServerHostName = if (blockManagerId.isDriver) {
237+
blockTransferService.hostName
238+
} else {
239+
conf.get("spark.shuffle.service.host", blockTransferService.hostName)
240+
}
241+
logInfo(s"external shuffle service host = $shuffleServerHostName, " +
242+
s"port = $externalShuffleServicePort")
243+
BlockManagerId(executorId, shuffleServerHostName, externalShuffleServicePort)
238244
} else {
239245
blockManagerId
240246
}

docs/running-on-kubernetes.md

Lines changed: 64 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -189,7 +189,7 @@ from the other deployment modes. See the [configuration page](configuration.html
189189
<td>
190190
The namespace that will be used for running the driver and executor pods. When using
191191
<code>spark-submit</code> in cluster mode, this can also be passed to <code>spark-submit</code> via the
192-
<code>--kubernetes-namespace</code> command line argument. The namespace must already exist.
192+
<code>--kubernetes-namespace</code> command line argument.
193193
</td>
194194
</tr>
195195
<tr>
@@ -208,6 +208,37 @@ from the other deployment modes. See the [configuration page](configuration.html
208208
<a href="https://docs.docker.com/engine/reference/commandline/tag/">Docker tag</a> format.
209209
</td>
210210
</tr>
211+
<tr>
212+
<td><code>spark.kubernetes.shuffle.namespace</code></td>
213+
<td><code>default</code></td>
214+
<td>
215+
Namespace in which the shuffle service pods are present. The shuffle service must be
216+
created in the cluster prior to attempts to use it.
217+
</td>
218+
</tr>
219+
<tr>
220+
<td><code>spark.kubernetes.shuffle.labels</code></td>
221+
<td><code>(none)</code></td>
222+
<td>
223+
Labels that will be used to look up shuffle service pods. This should be a comma-separated list of label key-value pairs,
224+
where each label is in the format <code>key=value</code>. The labels chosen must be such that
225+
they match exactly one shuffle service pod on each node that executors are launched.
226+
</td>
227+
</tr>
228+
<tr>
229+
<td><code>spark.kubernetes.allocation.batch.size</code></td>
230+
<td><code>5</code></td>
231+
<td>
232+
Number of pods to launch at once in each round of executor pod allocation.
233+
</td>
234+
</tr>
235+
<tr>
236+
<td><code>spark.kubernetes.allocation.batch.delay</code></td>
237+
<td><code>1</code></td>
238+
<td>
239+
Number of seconds to wait between each round of executor pod allocation.
240+
</td>
241+
</tr>
211242
<tr>
212243
<td><code>spark.kubernetes.authenticate.submission.caCertFile</code></td>
213244
<td>(none)</td>
@@ -389,10 +420,41 @@ from the other deployment modes. See the [configuration page](configuration.html
389420
</tr>
390421
</table>
391422

423+
## Dynamic Executor Scaling
424+
425+
Spark on Kubernetes supports Dynamic Allocation with cluster mode. This mode requires running
426+
an external shuffle service. This is typically a [daemonset](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/)
427+
with a provisioned [hostpath](https://kubernetes.io/docs/concepts/storage/volumes/#hostpath) volume.
428+
This shuffle service may be shared by executors belonging to different SparkJobs. Using Spark with dynamic allocation
429+
on Kubernetes assumes that a cluster administrator has set up one or more shuffle-service daemonsets in the cluster.
430+
431+
A sample configuration file is provided in `conf/kubernetes-shuffle-service.yaml` which can be customized as needed
432+
for a particular cluster. It is important to note that `spec.template.metadata.labels` are setup appropriately for the shuffle
433+
service because there may be multiple shuffle service instances running in a cluster. The labels give us a way to target a particular
434+
shuffle service.
435+
436+
For example, if the shuffle service we want to use is in the default namespace, and
437+
has pods with labels `app=spark-shuffle-service` and `spark-version=2.1.0`, we can
438+
use those tags to target that particular shuffle service at job launch time. In order to run a job with dynamic allocation enabled,
439+
the command may then look like the following:
440+
441+
bin/spark-submit \
442+
--deploy-mode cluster \
443+
--class org.apache.spark.examples.GroupByTest \
444+
--master k8s://<k8s-master>:<port> \
445+
--kubernetes-namespace default \
446+
--conf spark.app.name=group-by-test \
447+
--conf spark.kubernetes.driver.docker.image=kubespark/spark-driver:latest \
448+
--conf spark.kubernetes.executor.docker.image=kubespark/spark-executor:latest \
449+
--conf spark.dynamicAllocation.enabled=true \
450+
--conf spark.shuffle.service.enabled=true \
451+
--conf spark.kubernetes.shuffle.namespace=default \
452+
--conf spark.kubernetes.shuffle.labels="app=spark-shuffle-service,spark-version=2.1.0" \
453+
examples/jars/spark_examples_2.11-2.2.0.jar 10 400000 2
454+
392455
## Current Limitations
393456

394457
Running Spark on Kubernetes is currently an experimental feature. Some restrictions on the current implementation that
395458
should be lifted in the future include:
396-
* Applications can only use a fixed number of executors. Dynamic allocation is not supported.
397459
* Applications can only run in cluster mode.
398460
* Only Scala and Java applications can be run.

project/SparkBuild.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,7 @@ object BuildCommons {
5555
"core", "graphx", "mllib", "mllib-local", "repl", "network-common", "network-shuffle", "launcher", "unsafe",
5656
"tags", "sketch"
5757
).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects
58-
58+
5959
val optionallyEnabledProjects@Seq(mesos, yarn, java8Tests, sparkGangliaLgpl,
6060
streamingKinesisAsl, dockerIntegrationTests, kubernetes, _*) =
6161
Seq("mesos", "yarn", "java8-tests", "ganglia-lgpl", "streaming-kinesis-asl", "docker-integration-tests",

resource-managers/kubernetes/README.md

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -14,10 +14,10 @@ important matters to keep in mind when developing this feature.
1414

1515
# Building Spark with Kubernetes Support
1616

17-
To build Spark with Kubernetes support, use the `kubernetes` profile when invoking Maven.
17+
To build Spark with Kubernetes support, use the `kubernetes` profile when invoking Maven. For example, to simply compile
18+
the Kubernetes core implementation module along with its dependencies:
1819

19-
git checkout branch-2.1-kubernetes
20-
build/mvn package -Pkubernetes -DskipTests
20+
build/mvn compile -Pkubernetes -pl resource-managers/kubernetes/core -am -DskipTests
2121

2222
To build a distribution of Spark with Kubernetes support, use the `dev/make-distribution.sh` script, and add the
2323
`kubernetes` profile as part of the build arguments. Any other build arguments can be specified as one would expect when
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,41 @@
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+
package org.apache.spark.deploy.kubernetes
19+
20+
import org.apache.spark.SparkException
21+
22+
object ConfigurationUtils {
23+
def parseKeyValuePairs(
24+
maybeKeyValues: Option[String],
25+
configKey: String,
26+
keyValueType: String): Map[String, String] = {
27+
28+
maybeKeyValues.map(keyValues => {
29+
keyValues.split(",").map(_.trim).filterNot(_.isEmpty).map(keyValue => {
30+
keyValue.split("=", 2).toSeq match {
31+
case Seq(k, v) =>
32+
(k, v)
33+
case _ =>
34+
throw new SparkException(s"Custom $keyValueType set by $configKey must be a" +
35+
s" comma-separated list of key-value pairs, with format <key>=<value>." +
36+
s" Got value: $keyValue. All values: $keyValues")
37+
}
38+
}).toMap
39+
}).getOrElse(Map.empty[String, String])
40+
}
41+
}

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

Lines changed: 45 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -157,6 +157,13 @@ package object config extends Logging {
157157
.stringConf
158158
.createOptional
159159

160+
private[spark] val SPARK_SHUFFLE_SERVICE_HOST =
161+
ConfigBuilder("spark.shuffle.service.host")
162+
.doc("Host for Spark Shuffle Service")
163+
.internal()
164+
.stringConf
165+
.createOptional
166+
160167
// Note that while we set a default for this when we start up the
161168
// scheduler, the specific default value is dynamically determined
162169
// based on the executor memory.
@@ -270,6 +277,44 @@ package object config extends Logging {
270277
.stringConf
271278
.createOptional
272279

280+
private[spark] val KUBERNETES_SHUFFLE_NAMESPACE =
281+
ConfigBuilder("spark.kubernetes.shuffle.namespace")
282+
.doc("Namespace of the shuffle service")
283+
.stringConf
284+
.createWithDefault("default")
285+
286+
private[spark] val KUBERNETES_SHUFFLE_SVC_IP =
287+
ConfigBuilder("spark.kubernetes.shuffle.ip")
288+
.doc("This setting is for debugging only. Setting this " +
289+
"allows overriding the IP that the executor thinks its colocated " +
290+
"shuffle service is on")
291+
.stringConf
292+
.createOptional
293+
294+
private[spark] val KUBERNETES_SHUFFLE_LABELS =
295+
ConfigBuilder("spark.kubernetes.shuffle.labels")
296+
.doc("Labels to identify the shuffle service")
297+
.stringConf
298+
.createOptional
299+
300+
private[spark] val KUBERNETES_SHUFFLE_DIR =
301+
ConfigBuilder("spark.kubernetes.shuffle.dir")
302+
.doc("Path to the shared shuffle directories.")
303+
.stringConf
304+
.createOptional
305+
306+
private[spark] val KUBERNETES_ALLOCATION_BATCH_SIZE =
307+
ConfigBuilder("spark.kubernetes.allocation.batch.size")
308+
.doc("Number of pods to launch at once in each round of dynamic allocation. ")
309+
.intConf
310+
.createWithDefault(5)
311+
312+
private[spark] val KUBERNETES_ALLOCATION_BATCH_DELAY =
313+
ConfigBuilder("spark.kubernetes.allocation.batch.delay")
314+
.doc("Number of seconds to wait between each round of executor allocation. ")
315+
.longConf
316+
.createWithDefault(1)
317+
273318
private[spark] val DRIVER_SERVICE_MANAGER_TYPE =
274319
ConfigBuilder("spark.kubernetes.driver.serviceManagerType")
275320
.doc("A tag indicating which class to use for creating the Kubernetes service and" +

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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -113,4 +113,5 @@ package object constants {
113113
s"$INIT_CONTAINER_PROPERTIES_FILE_MOUNT_PATH/$INIT_CONTAINER_PROPERTIES_FILE_NAME"
114114
private[spark] val DOWNLOAD_JARS_VOLUME_NAME = "download-jars"
115115
private[spark] val DOWNLOAD_FILES_VOLUME_NAME = "download-files"
116+
private[spark] val DEFAULT_SHUFFLE_MOUNT_NAME = "shuffle"
116117
}

0 commit comments

Comments
 (0)