-
Notifications
You must be signed in to change notification settings - Fork 28k
/
ExecutorPodsAllocator.scala
134 lines (117 loc) · 5.84 KB
/
ExecutorPodsAllocator.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
/*
* 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.util.concurrent.atomic.{AtomicInteger, AtomicLong}
import io.fabric8.kubernetes.api.model.PodBuilder
import io.fabric8.kubernetes.client.KubernetesClient
import scala.collection.mutable
import org.apache.spark.{SparkConf, SparkException}
import org.apache.spark.deploy.k8s.Config._
import org.apache.spark.deploy.k8s.Constants._
import org.apache.spark.deploy.k8s.KubernetesConf
import org.apache.spark.internal.Logging
import org.apache.spark.util.{Clock, Utils}
private[spark] class ExecutorPodsAllocator(
conf: SparkConf,
executorBuilder: KubernetesExecutorBuilder,
kubernetesClient: KubernetesClient,
snapshotsStore: ExecutorPodsSnapshotsStore,
clock: Clock) extends Logging {
private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
private val totalExpectedExecutors = new AtomicInteger(0)
private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
private val podAllocationDelay = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
private val podCreationTimeout = math.max(podAllocationDelay * 5, 60000)
private val kubernetesDriverPodName = conf
.get(KUBERNETES_DRIVER_POD_NAME)
.getOrElse(throw new SparkException("Must specify the driver pod name"))
private val driverPod = kubernetesClient.pods()
.withName(kubernetesDriverPodName)
.get()
// Executor IDs that have been requested from Kubernetes but have not been detected in any
// snapshot yet. Mapped to the timestamp when they were created.
private val newlyCreatedExecutors = mutable.Map.empty[Long, Long]
def start(applicationId: String): Unit = {
snapshotsStore.addSubscriber(podAllocationDelay) {
processSnapshot(applicationId, _)
}
}
def setTotalExpectedExecutors(total: Int): Unit = totalExpectedExecutors.set(total)
private def processSnapshot(applicationId: String, snapshot: ExecutorPodsSnapshot): Unit = {
snapshot.executorPods.keys.foreach { newlyCreatedExecutors -= _ }
// For all executors we've created against the API but have not seen in a snapshot
// yet - check the current time. If the current time has exceeded some threshold,
// assume that the pod was either never created (the API server never properly
// handled the creation request), or the API server created the pod but we missed
// both the creation and deletion events. In either case, delete the missing pod
// if possible, and mark such a pod to be rescheduled below.
(newlyCreatedExecutors.keySet -- snapshot.executorPods.keySet).foreach { execId =>
if (clock.getTimeMillis() - newlyCreatedExecutors(execId) > podCreationTimeout) {
logWarning(s"Executor with id $execId was not detected in the Kubernetes" +
s" cluster after $podCreationTimeout milliseconds despite the fact that a" +
" previous allocation attempt tried to create it. The executor may have been" +
" deleted but the application missed the deletion event.")
Utils.tryLogNonFatalError {
kubernetesClient
.pods()
.withLabel(SPARK_EXECUTOR_ID_LABEL, execId.toString)
.delete()
}
newlyCreatedExecutors -= execId
}
}
val currentRunningExecutors = snapshot.executorPods.values.count {
case PodRunning(_) => true
case _ => false
}
val currentPendingExecutors = snapshot.executorPods.values.count {
case PodPending(_) => true
case _ => false
}
val currentTotalExpectedExecutors = totalExpectedExecutors.get
if (newlyCreatedExecutors.isEmpty
&& currentPendingExecutors == 0
&& currentRunningExecutors < currentTotalExpectedExecutors) {
val numExecutorsToAllocate = math.min(
currentTotalExpectedExecutors - currentRunningExecutors, podAllocationSize)
logInfo(s"Going to request $numExecutorsToAllocate executors from Kubernetes.")
for ( _ <- 0 until numExecutorsToAllocate) {
val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet()
val executorConf = KubernetesConf.createExecutorConf(
conf,
newExecutorId.toString,
applicationId,
driverPod)
val executorPod = executorBuilder.buildFromFeatures(executorConf)
val podWithAttachedContainer = new PodBuilder(executorPod.pod)
.editOrNewSpec()
.addToContainers(executorPod.container)
.endSpec()
.build()
kubernetesClient.pods().create(podWithAttachedContainer)
newlyCreatedExecutors(newExecutorId) = clock.getTimeMillis()
}
} else if (currentRunningExecutors >= currentTotalExpectedExecutors) {
// TODO handle edge cases if we end up with more running executors than expected.
logDebug("Current number of running executors is equal to the number of requested" +
" executors. Not scaling up further.")
} else if (newlyCreatedExecutors.nonEmpty || currentPendingExecutors != 0) {
logDebug(s"Still waiting for ${newlyCreatedExecutors.size + currentPendingExecutors}" +
s" executors to begin running before requesting for more executors.")
}
}
}