Github user markhamstra commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r151259662
  
    --- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
    @@ -0,0 +1,427 @@
    +/*
    + * 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 javax.annotation.concurrent.GuardedBy
    +
    +import io.fabric8.kubernetes.api.model._
    +import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
    +import io.fabric8.kubernetes.client.Watcher.Action
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +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, 
SchedulerBackendUtils}
    +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
    +  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
    +  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
    +  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 = kubernetesClient.pods()
    +    .inNamespace(kubernetesNamespace)
    +    .withName(kubernetesDriverPodName)
    +    .get()
    +
    +  override val minRegisteredRatio =
    +    if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  protected val 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 = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
    +
    +  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
    +
    +  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
    +
    +  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()
    +      val executorsToAllocate = mutable.Map[String, Pod]()
    +      val currentTotalRegisteredExecutors = totalRegisteredExecutors.get
    +      val currentTotalExpectedExecutors = totalExpectedExecutors.get
    +      val currentNodeToLocalTaskCount = getNodesWithLocalTaskCounts
    +      if (currentTotalRegisteredExecutors < runningExecutorsToPods.size) {
    +        logDebug("Waiting for pending executors before scaling")
    +      } else if (currentTotalExpectedExecutors <= 
runningExecutorsToPods.size) {
    +        logDebug("Maximum allowed executor limit reached. Not scaling up 
further.")
    +      } else {
    +        for (i <- 0 until math.min(
    +          currentTotalExpectedExecutors - runningExecutorsToPods.size, 
podAllocationSize)) {
    +          val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString
    +          val executorPod = executorPodFactory.createExecutorPod(
    +            executorId,
    +            applicationId(),
    +            driverUrl,
    +            conf.getExecutorEnv,
    +            driverPod,
    +            currentNodeToLocalTaskCount)
    +          
require(executorPod.getMetadata.getLabels.containsKey(SPARK_EXECUTOR_ID_LABEL),
    +            s"Illegal internal state for pod with name 
${executorPod.getMetadata.getName} - all" +
    +              s" executor pods must contain the label 
$SPARK_EXECUTOR_ID_LABEL.")
    +          val resolvedExecutorIdLabel = 
executorPod.getMetadata.getLabels.get(
    +            SPARK_EXECUTOR_ID_LABEL)
    +          require(resolvedExecutorIdLabel == executorId,
    +            s"Illegal internal state for pod with name 
${executorPod.getMetadata.getName} - all" +
    +              s" executor pods must map the label with key 
${SPARK_EXECUTOR_ID_LABEL} to the" +
    +              s" executor's ID. This label mapped instead to: 
$resolvedExecutorIdLabel.")
    +          executorsToAllocate(executorId) = executorPod
    +          logInfo(
    +            s"Requesting a new executor, total executors is now 
${runningExecutorsToPods.size}")
    +        }
    +      }
    +      val allocatedExecutors = executorsToAllocate.mapValues { pod =>
    +        Utils.tryLog {
    +          kubernetesClient.pods().create(pod)
    +        }
    +      }
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        allocatedExecutors.map {
    +          case (executorId, attemptedAllocatedExecutor) =>
    +            attemptedAllocatedExecutor.map { successfullyAllocatedExecutor 
=>
    +              runningExecutorsToPods.put(executorId, 
successfullyAllocatedExecutor)
    +            }
    +        }
    +      }
    +    }
    +
    +    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.asScala.foreach {
    +        case (executorId, executorPod) =>
    +          val knownExitReason = Option(podsWithKnownExitReasons.remove(
    +            executorPod.getMetadata.getName))
    +          knownExitReason.fold {
    +            removeExecutorOrIncrementLossReasonCheckCount(executorId)
    +          } { executorExited =>
    +            logWarning(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) {
    +              logInfo(s"Executor $executorId exited because of the 
application.")
    +              deleteExecutorFromDataStructures(executorId)
    +            } else {
    +              logInfo(s"Executor $executorId failed because of a framework 
error.")
    +              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 = {
    +      deleteExecutorFromDataStructures(executorId)
    +        .foreach(pod => kubernetesClient.pods().delete(pod))
    +    }
    +
    +    def deleteExecutorFromDataStructures(executorId: String): Option[Pod] 
= {
    +      disconnectedPodsByExecutorIdPendingRemoval.remove(executorId)
    +      executorReasonCheckAttemptCounts -= executorId
    +      podsWithKnownExitReasons.remove(executorId)
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        runningExecutorsToPods.remove(executorId).orElse {
    +          logWarning(s"Unable to remove pod for unknown executor 
$executorId")
    +          None
    +        }
    +      }
    +    }
    +  }
    +
    +  override def sufficientResourcesRegistered(): Boolean = {
    +    totalRegisteredExecutors.get() >= initialExecutors * minRegisteredRatio
    +  }
    +
    +  override def start(): Unit = {
    +    super.start()
    +    executorWatchResource.set(
    +      kubernetesClient
    +        .pods()
    +        .withLabel(SPARK_APP_ID_LABEL, applicationId())
    +        .watch(new ExecutorPodsWatcher()))
    +
    +    allocatorExecutor.scheduleWithFixedDelay(
    +      allocatorRunnable, 0L, podAllocationInterval, TimeUnit.SECONDS)
    +
    +    if (!Utils.isDynamicAllocationEnabled(conf)) {
    +      doRequestTotalExecutors(initialExecutors)
    +    }
    +  }
    +
    +  override def stop(): Unit = {
    +    // stop allocation of new resources and caches.
    +    allocatorExecutor.shutdown()
    +
    +    // send stop message to executors so they shut down cleanly
    +    super.stop()
    +
    +    // then delete the executor pods
    +    Utils.tryLogNonFatalError {
    +      val executorPodsToDelete = RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        val runningExecutorPodsCopy = 
Seq(runningExecutorsToPods.values.toSeq: _*)
    +        runningExecutorsToPods.clear()
    +        runningExecutorPodsCopy
    +      }
    +      kubernetesClient.pods().delete(executorPodsToDelete: _*)
    +      executorPodsByIPs.clear()
    +      val resource = executorWatchResource.getAndSet(null)
    +      if (resource != null) {
    +        resource.close()
    +      }
    +    }
    +    Utils.tryLogNonFatalError {
    +      logInfo("Closing kubernetes client")
    +      kubernetesClient.close()
    +    }
    +  }
    +
    +  /**
    +   * @return A map of K8s cluster nodes to the number of tasks that could 
benefit from data
    +   *         locality if an executor launches on the cluster node.
    +   */
    +  private def getNodesWithLocalTaskCounts() : Map[String, Int] = {
    +    val nodeToLocalTaskCount = mutable.Map[String, Int]() ++
    +      synchronized {
    +        hostToLocalTaskCount
    +      }
    +    for (pod <- executorPodsByIPs.values().asScala) {
    +      // Remove cluster nodes that are running our executors already.
    +      // TODO: This prefers spreading out executors across nodes. In case 
users want
    +      // consolidating executors on fewer nodes, introduce a flag. See the 
spark.deploy.spreadOut
    +      // flag that Spark standalone has: 
https://spark.apache.org/docs/latest/spark-standalone.html
    +      nodeToLocalTaskCount.remove(pod.getSpec.getNodeName).nonEmpty ||
    +        nodeToLocalTaskCount.remove(pod.getStatus.getHostIP).nonEmpty ||
    +        nodeToLocalTaskCount.remove(
    +          
InetAddress.getByName(pod.getStatus.getHostIP).getCanonicalHostName).nonEmpty
    +    }
    +    nodeToLocalTaskCount.toMap[String, Int]
    +  }
    +
    +  override def doRequestTotalExecutors(requestedTotal: Int): 
Future[Boolean] = Future[Boolean] {
    +    totalExpectedExecutors.set(requestedTotal)
    +    true
    +  }
    +
    +  override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] 
= Future[Boolean] {
    +    val podsToDelete = mutable.Buffer[Pod]()
    +    RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    --- End diff --
    
    Oh no! This can of worms again! :)
    
    Lets just say that opinions vary both across the Scala community and within 
Spark development as to the best or most proper way to handle Options when you 
want to do something in both the Some and None cases. Within Spark code, use of 
`fold` with an Option is not allowed. As for other alternatives, we have no 
consistent rules or practice.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to