/
KubernetesClusterManager.scala
149 lines (131 loc) · 5.99 KB
/
KubernetesClusterManager.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.scheduler.cluster.k8s
import java.io.File
import java.util.concurrent.TimeUnit
import com.google.common.cache.CacheBuilder
import io.fabric8.kubernetes.client.Config
import org.apache.spark.SparkContext
import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkKubernetesClientFactory}
import org.apache.spark.deploy.k8s.Config._
import org.apache.spark.deploy.k8s.Constants.DEFAULT_EXECUTOR_CONTAINER_NAME
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
import org.apache.spark.util.{SystemClock, ThreadUtils}
private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging {
override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s")
override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = {
new TaskSchedulerImpl(sc)
}
override def createSchedulerBackend(
sc: SparkContext,
masterURL: String,
scheduler: TaskScheduler): SchedulerBackend = {
val wasSparkSubmittedInClusterMode = sc.conf.get(KUBERNETES_DRIVER_SUBMIT_CHECK)
val (authConfPrefix,
apiServerUri,
defaultServiceAccountToken,
defaultServiceAccountCaCrt) = if (wasSparkSubmittedInClusterMode) {
require(sc.conf.get(KUBERNETES_DRIVER_POD_NAME).isDefined,
"If the application is deployed using spark-submit in cluster mode, the driver pod name " +
"must be provided.")
val serviceAccountToken =
Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)).filter(_.exists)
val serviceAccountCaCrt =
Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH)).filter(_.exists)
(KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX,
sc.conf.get(KUBERNETES_DRIVER_MASTER_URL),
serviceAccountToken,
serviceAccountCaCrt)
} else {
(KUBERNETES_AUTH_CLIENT_MODE_PREFIX,
KubernetesUtils.parseMasterUrl(masterURL),
None,
None)
}
// If KUBERNETES_EXECUTOR_POD_NAME_PREFIX is not set, initialize it so that all executors have
// the same prefix. This is needed for client mode, where the feature steps code that sets this
// configuration is not used.
//
// If/when feature steps are executed in client mode, they should instead take care of this,
// and this code should be removed.
if (!sc.conf.contains(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)) {
sc.conf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX,
KubernetesConf.getResourceNamePrefix(sc.conf.get("spark.app.name")))
}
val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient(
apiServerUri,
Some(sc.conf.get(KUBERNETES_NAMESPACE)),
authConfPrefix,
SparkKubernetesClientFactory.ClientType.Driver,
sc.conf,
defaultServiceAccountToken,
defaultServiceAccountCaCrt)
if (sc.conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).isDefined) {
KubernetesUtils.loadPodFromTemplate(
kubernetesClient,
new File(sc.conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).get),
sc.conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_CONTAINER_NAME))
}
val schedulerExecutorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
"kubernetes-executor-maintenance")
ExecutorPodsSnapshot.setShouldCheckAllContainers(
sc.conf.get(KUBERNETES_EXECUTOR_CHECK_ALL_CONTAINERS))
val sparkContainerName = sc.conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_CONTAINER_NAME)
.getOrElse(DEFAULT_EXECUTOR_CONTAINER_NAME)
ExecutorPodsSnapshot.setSparkContainerName(sparkContainerName)
val subscribersExecutor = ThreadUtils
.newDaemonThreadPoolScheduledExecutor(
"kubernetes-executor-snapshots-subscribers", 2)
val snapshotsStore = new ExecutorPodsSnapshotsStoreImpl(subscribersExecutor)
val removedExecutorsCache = CacheBuilder.newBuilder()
.expireAfterWrite(3, TimeUnit.MINUTES)
.build[java.lang.Long, java.lang.Long]()
val executorPodsLifecycleEventHandler = new ExecutorPodsLifecycleManager(
sc.conf,
kubernetesClient,
snapshotsStore,
removedExecutorsCache)
val executorPodsAllocator = new ExecutorPodsAllocator(
sc.conf,
sc.env.securityManager,
new KubernetesExecutorBuilder(),
kubernetesClient,
snapshotsStore,
new SystemClock())
val podsWatchEventSource = new ExecutorPodsWatchSnapshotSource(
snapshotsStore,
kubernetesClient)
val eventsPollingExecutor = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
"kubernetes-executor-pod-polling-sync")
val podsPollingEventSource = new ExecutorPodsPollingSnapshotSource(
sc.conf, kubernetesClient, snapshotsStore, eventsPollingExecutor)
new KubernetesClusterSchedulerBackend(
scheduler.asInstanceOf[TaskSchedulerImpl],
sc,
kubernetesClient,
schedulerExecutorService,
snapshotsStore,
executorPodsAllocator,
executorPodsLifecycleEventHandler,
podsWatchEventSource,
podsPollingEventSource)
}
override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = {
scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend)
}
}