Github user foxish commented on a diff in the pull request:
https://github.com/apache/spark/pull/19468#discussion_r146437447
--- Diff:
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
---
@@ -0,0 +1,456 @@
+/*
+ * 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.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService,
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong,
AtomicReference}
+
+import scala.collection.{concurrent, mutable}
+import scala.collection.JavaConverters._
+import scala.concurrent.{ExecutionContext, Future}
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient,
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.config._
+import org.apache.spark.deploy.k8s.constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost,
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+ scheduler: TaskSchedulerImpl,
+ rpcEnv: RpcEnv,
+ executorPodFactory: ExecutorPodFactory,
+ kubernetesClient: KubernetesClient,
+ allocatorExecutor: ScheduledExecutorService,
+ requestExecutorsService: ExecutorService)
+ extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+ import KubernetesClusterSchedulerBackend._
+
+ private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+ private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+ // Indexed by executor IDs and guarded by RUNNING_EXECUTOR_PODS_LOCK.
+ private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+ // Indexed by executor pod names and guarded by
RUNNING_EXECUTOR_PODS_LOCK.
+ private val runningPodsToExecutors = new mutable.HashMap[String, String]
+ private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+ private val podsWithKnownExitReasons = new ConcurrentHashMap[String,
ExecutorExited]()
+ private val disconnectedPodsByExecutorIdPendingRemoval = new
ConcurrentHashMap[String, Pod]()
+
+ private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+ private val kubernetesDriverPodName = conf
+ .get(KUBERNETES_DRIVER_POD_NAME)
+ .getOrElse(
+ throw new SparkException("Must specify the driver pod name"))
+ private implicit val requestExecutorContext =
ExecutionContext.fromExecutorService(
+ requestExecutorsService)
+
+ private val driverPod = try {
+ kubernetesClient.pods()
+ .inNamespace(kubernetesNamespace)
+ .withName(kubernetesDriverPodName)
+ .get()
+ } catch {
+ case throwable: Throwable =>
+ logError(s"Executor cannot find driver pod.", throwable)
+ throw new SparkException(s"Executor cannot find driver pod",
throwable)
+ }
+
+ override val minRegisteredRatio =
+ if
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+ 0.8
+ } else {
+ super.minRegisteredRatio
+ }
+
+ private val executorWatchResource = new AtomicReference[Closeable]
+ protected var totalExpectedExecutors = new AtomicInteger(0)
+
+ private val driverUrl = RpcEndpointAddress(
+ conf.get("spark.driver.host"),
+ conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+ CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+ private val initialExecutors = getInitialTargetExecutorNumber()
+
+ private val podAllocationInterval =
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+ require(podAllocationInterval > 0, s"Allocation batch delay " +
+ s"${KUBERNETES_ALLOCATION_BATCH_DELAY} " +
+ s"is ${podAllocationInterval}, should be a positive integer")
+
+ private val podAllocationSize =
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+ require(podAllocationSize > 0, s"Allocation batch size " +
+ s"${KUBERNETES_ALLOCATION_BATCH_SIZE} " +
+ s"is ${podAllocationSize}, should be a positive integer")
+
+ private val allocatorRunnable = new Runnable {
+
+ // Maintains a map of executor id to count of checks performed to
learn the loss reason
+ // for an executor.
+ private val executorReasonCheckAttemptCounts = new
mutable.HashMap[String, Int]
+
+ override def run(): Unit = {
+ handleDisconnectedExecutors()
+ RUNNING_EXECUTOR_PODS_LOCK.synchronized {
+ if (totalRegisteredExecutors.get() < runningExecutorsToPods.size) {
+ logDebug("Waiting for pending executors before scaling")
+ } else if (totalExpectedExecutors.get() <=
runningExecutorsToPods.size) {
+ logDebug("Maximum allowed executor limit reached. Not scaling up
further.")
+ } else {
+ val nodeToLocalTaskCount = getNodesWithLocalTaskCounts
+ for (i <- 0 until math.min(
+ totalExpectedExecutors.get - runningExecutorsToPods.size,
podAllocationSize)) {
+ val (executorId, pod) =
allocateNewExecutorPod(nodeToLocalTaskCount)
+ runningExecutorsToPods.put(executorId, pod)
+ runningPodsToExecutors.put(pod.getMetadata.getName, executorId)
+ logInfo(
+ s"Requesting a new executor, total executors is now
${runningExecutorsToPods.size}")
+ }
+ }
+ }
+ }
+
+ def handleDisconnectedExecutors(): Unit = {
+ // For each disconnected executor, synchronize with the loss reasons
that may have been found
+ // by the executor pod watcher. If the loss reason was discovered by
the watcher,
+ // inform the parent class with removeExecutor.
+ disconnectedPodsByExecutorIdPendingRemoval.keys().asScala.foreach {
case (executorId) =>
+ val executorPod =
disconnectedPodsByExecutorIdPendingRemoval.get(executorId)
+ val knownExitReason = Option(podsWithKnownExitReasons.remove(
+ executorPod.getMetadata.getName))
+ knownExitReason.fold {
+ removeExecutorOrIncrementLossReasonCheckCount(executorId)
+ } { executorExited =>
+ logDebug(s"Removing executor $executorId with loss reason " +
executorExited.message)
+ removeExecutor(executorId, executorExited)
+ // We keep around executors that have exit conditions caused by
the application. This
+ // allows them to be debugged later on. Otherwise, mark them as
to be deleted from the
+ // the API server.
+ if (!executorExited.exitCausedByApp) {
+ deleteExecutorFromClusterAndDataStructures(executorId)
+ }
+ }
+ }
+ }
+
+ def removeExecutorOrIncrementLossReasonCheckCount(executorId: String):
Unit = {
+ val reasonCheckCount =
executorReasonCheckAttemptCounts.getOrElse(executorId, 0)
+ if (reasonCheckCount >= MAX_EXECUTOR_LOST_REASON_CHECKS) {
+ removeExecutor(executorId, SlaveLost("Executor lost for unknown
reasons."))
+ deleteExecutorFromClusterAndDataStructures(executorId)
+ } else {
+ executorReasonCheckAttemptCounts.put(executorId, reasonCheckCount
+ 1)
+ }
+ }
+
+ def deleteExecutorFromClusterAndDataStructures(executorId: String):
Unit = {
+ disconnectedPodsByExecutorIdPendingRemoval.remove(executorId)
+ executorReasonCheckAttemptCounts -= executorId
+ RUNNING_EXECUTOR_PODS_LOCK.synchronized {
+ runningExecutorsToPods.remove(executorId).map { pod =>
+ kubernetesClient.pods().delete(pod)
+ runningPodsToExecutors.remove(pod.getMetadata.getName)
+ }.getOrElse(logWarning(s"Unable to remove pod for unknown executor
$executorId"))
+ }
+ }
+ }
+
+ private def getInitialTargetExecutorNumber(defaultNumExecutors: Int =
1): Int = {
+ if (Utils.isDynamicAllocationEnabled(conf)) {
+ val minNumExecutors =
conf.getInt("spark.dynamicAllocation.minExecutors", 0)
+ val initialNumExecutors =
Utils.getDynamicAllocationInitialExecutors(conf)
+ val maxNumExecutors =
conf.getInt("spark.dynamicAllocation.maxExecutors", 1)
+ require(initialNumExecutors >= minNumExecutors &&
initialNumExecutors <= maxNumExecutors,
+ s"initial executor number $initialNumExecutors must between min
executor number " +
+ s"$minNumExecutors and max executor number $maxNumExecutors")
+
+ initialNumExecutors
+ } else {
+ conf.getInt("spark.executor.instances", defaultNumExecutors)
+ }
+
+ }
+
+ override def applicationId(): String = conf.get("spark.app.id",
super.applicationId())
--- End diff --
Client mode on k8s will look very different from that on other cluster
managers. This is because in almost all k8s clusters, networking in setup such
that pods (or simply containers) within clusters are not routable directly from
outside, and vice versa.
An alternative to support the same functionality is ["in-cluster client
mode"](https://github.com/apache-spark-on-k8s/spark/pull/456). This will look
similar to cluster mode (driver and executor will run within the cluster),
except that we'd set a couple of additional parameters on the driver pod, to
allow someone outside the cluster to connect with it interactively (using the
Kubernetes APIServer as a proxy). We don't foresee a design in which the
`KubernetesClusterSchedulerBackend` will be invoked directly on the
client-side, because there is no standard mechanism by which executors can
reach back out to the driver in k8s (this is by design, and the reasoning is
explained in part by the requirements in
https://kubernetes.io/docs/concepts/cluster-administration/networking/#kubernetes-model).
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]