You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by foxish <gi...@git.apache.org> on 2017/10/11 00:16:56 UTC

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

GitHub user foxish opened a pull request:

    https://github.com/apache/spark/pull/19468

    [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Scheduler Backend

    ## What changes were proposed in this pull request?
    
    This change adds in a stripped down version of the `KubernetesClusterSchedulerBackend` for Spark.
    It's step 1 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935).
    This addition is covered by the [SPIP vote](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) which passed on Aug 31 .
    
    ## How was this patch tested?
    
    - The patch contains unit tests which are passing.
    - Manual testing: `./build/mvn -Pkubernetes clean package` succeeded.
    - It is a **subset** of the entire changelist hosted in http://github.com/apache-spark-on-k8s/spark which is under active use in several organizations.
    - There is integration testing enabled in the fork currently [hosted by PepperData](spark-k8s-jenkins.pepperdata.org:8080) which is being moved over to RiseLAB CI.
    - Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/apache-spark-on-k8s/spark spark-kubernetes-3

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/19468.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #19468
    
----
commit f6fdd6a40b38b413cd55e2315dd9fb5bc6ca575c
Author: foxish <ra...@google.com>
Date:   2017-09-15T03:10:24Z

    Spark on Kubernetes - basic scheduler backend

----


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150933856
  
    --- 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,
    --- End diff --
    
    I disagree with removing the checks here. The contract that the labels are present are in ExecutorPodFactory, which is further removed from this class. It would be good for the contract to be explicit here, because we can then catch errors in integration tests and diagnose them quickly.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153331617
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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()
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        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)
    +            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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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 = synchronized {
    +      mutable.Map[String, Int]() ++ 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 = RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +      executorIds.flatMap { executorId =>
    +        runningExecutorsToPods.remove(executorId) match {
    +          case Some(pod) =>
    +            disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            Some(pod)
    +
    +          case None =>
    +            logWarning(s"Unable to remove pod for unknown executor $executorId")
    +            None
    +        }
    +      }
    +    }
    +
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      val podName = pod.getMetadata.getName
    +      val podIP = pod.getStatus.getPodIP
    +
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +            && pod.getMetadata.getDeletionTimestamp == null) =>
    +          val clusterNodeName = pod.getSpec.getNodeName
    +          logInfo(s"Executor pod $podName ready, launched at $clusterNodeName as IP $podIP.")
    +          executorPodsByIPs.put(podIP, pod)
    +
    +        case Action.DELETED | Action.ERROR =>
    +          val executorId = getExecutorId(pod)
    +          logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +          if (podIP != null) {
    +            executorPodsByIPs.remove(podIP)
    +          }
    +
    +          val executorExitReason = if (action == Action.ERROR) {
    +            logWarning(s"Received error event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnError(pod)
    +          } else if (action == Action.DELETED) {
    +            logWarning(s"Received delete event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnDelete(pod)
    +          } else {
    +            throw new IllegalStateException(
    +              s"Unknown action that should only be DELETED or ERROR: $action")
    +          }
    +          podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason)
    +
    +          if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) {
    +            log.warn(s"Executor with id $executorId was not marked as disconnected, but the" +
    +              s" watch received an event of type $action for this executor. The executor may" +
    +              " have failed to start in the first place and never registered with the driver.")
    +          }
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +
    +        case _ => logDebug(s"Received event of executor pod $podName: " + action)
    +      }
    +    }
    +
    +    override def onClose(cause: KubernetesClientException): Unit = {
    +      logDebug("Executor pod watch closed.", cause)
    +    }
    +
    +    private def getExecutorExitStatus(pod: Pod): Int = {
    +      val containerStatuses = pod.getStatus.getContainerStatuses
    +      if (!containerStatuses.isEmpty) {
    +        // we assume the first container represents the pod status. This assumption may not hold
    +        // true in the future. Revisit this if side-car containers start running inside executor
    +        // pods.
    +        getExecutorExitStatus(containerStatuses.get(0))
    +      } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS
    +    }
    +
    +    private def getExecutorExitStatus(containerStatus: ContainerStatus): Int = {
    +      Option(containerStatus.getState).map { containerState =>
    +        Option(containerState.getTerminated).map { containerStateTerminated =>
    +          containerStateTerminated.getExitCode.intValue()
    +        }.getOrElse(UNKNOWN_EXIT_CODE)
    +      }.getOrElse(UNKNOWN_EXIT_CODE)
    +    }
    +
    +    private def isPodAlreadyReleased(pod: Pod): Boolean = {
    +      val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL)
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        !runningExecutorsToPods.contains(executorId)
    +      }
    +    }
    +
    +    private def executorExitReasonOnError(pod: Pod): ExecutorExited = {
    +      val containerExitStatus = getExecutorExitStatus(pod)
    +      // container was probably actively killed by the driver.
    +      if (isPodAlreadyReleased(pod)) {
    +        ExecutorExited(containerExitStatus, exitCausedByApp = false,
    +          s"Container in pod ${pod.getMetadata.getName} exited from explicit termination" +
    +            " request.")
    +      } else {
    +        val containerExitReason = s"Pod ${pod.getMetadata.getName}'s executor container " +
    +          s"exited with exit status code $containerExitStatus."
    +        ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason)
    +      }
    +    }
    +
    +    private def executorExitReasonOnDelete(pod: Pod): ExecutorExited = {
    +      val exitMessage = if (isPodAlreadyReleased(pod)) {
    +        s"Container in pod ${pod.getMetadata.getName} exited from explicit termination request."
    +      } else {
    +        s"Pod ${pod.getMetadata.getName} deleted or lost."
    +      }
    +      ExecutorExited(getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage)
    +    }
    +
    +    private def getExecutorId(pod: Pod): String = {
    +      val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL)
    +      require(executorId != null, "Unexpected pod metadata; expected all executor pods " +
    +        s"to have label $SPARK_EXECUTOR_ID_LABEL.")
    +      executorId
    +    }
    +  }
    +
    +  override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = {
    +    new KubernetesDriverEndpoint(rpcEnv, properties)
    +  }
    +
    +  private class KubernetesDriverEndpoint(
    +      rpcEnv: RpcEnv,
    +      sparkProperties: Seq[(String, String)])
    +    extends DriverEndpoint(rpcEnv, sparkProperties) {
    +
    +    override def onDisconnected(rpcAddress: RpcAddress): Unit = {
    +      addressToExecutorId.get(rpcAddress).foreach { executorId =>
    +        if (disableExecutor(executorId)) {
    +          RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +            runningExecutorsToPods.get(executorId).foreach { pod =>
    +              disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            }
    +          }
    +        }
    +      }
    +    }
    +  }
    +}
    +
    +private object KubernetesClusterSchedulerBackend {
    +  private val UNKNOWN_EXIT_CODE = -1
    +  // Number of times we are allowed check for the loss reason for an executor before we give up
    +  // and assume the executor failed for good, and attribute it to a framework fault.
    +  val MAX_EXECUTOR_LOST_REASON_CHECKS = 10
    --- End diff --
    
    This is getting pushed from remote server, right ?
    Which effectively means there can be arbitrary delays - either due to remote server, local node, networking infra in between - or something else. We cannot assume quick turnaround.
    In addition, as I mentioned above, batch delay is user configured - and can be set aggresively by user.
    
    Not that I am necessarily advocating for customization - if it is a smart default which will never be breached, it is fine ! I want to understand if it is.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152458539
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    --- End diff --
    
    We are already translating it to k8s specific setting at line 117. For reference:
    
    ```
    val executorMemoryQuantity = new QuantityBuilder(false)
          .withAmount(s"${executorMemoryMiB}Mi")
    ```


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #84233 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84233/testReport)** for PR 19468 at commit [`4bed817`](https://github.com/apache/spark/commit/4bed817e5ab1a26e70050254fec453e839af4c8c).


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153335335
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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()
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        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)
    +            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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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 = synchronized {
    +      mutable.Map[String, Int]() ++ 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 = RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +      executorIds.flatMap { executorId =>
    +        runningExecutorsToPods.remove(executorId) match {
    +          case Some(pod) =>
    +            disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            Some(pod)
    +
    +          case None =>
    +            logWarning(s"Unable to remove pod for unknown executor $executorId")
    +            None
    +        }
    +      }
    +    }
    +
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      val podName = pod.getMetadata.getName
    +      val podIP = pod.getStatus.getPodIP
    +
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +            && pod.getMetadata.getDeletionTimestamp == null) =>
    +          val clusterNodeName = pod.getSpec.getNodeName
    +          logInfo(s"Executor pod $podName ready, launched at $clusterNodeName as IP $podIP.")
    +          executorPodsByIPs.put(podIP, pod)
    +
    +        case Action.DELETED | Action.ERROR =>
    +          val executorId = getExecutorId(pod)
    +          logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +          if (podIP != null) {
    +            executorPodsByIPs.remove(podIP)
    +          }
    +
    +          val executorExitReason = if (action == Action.ERROR) {
    +            logWarning(s"Received error event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnError(pod)
    +          } else if (action == Action.DELETED) {
    +            logWarning(s"Received delete event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnDelete(pod)
    +          } else {
    +            throw new IllegalStateException(
    +              s"Unknown action that should only be DELETED or ERROR: $action")
    +          }
    +          podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason)
    +
    +          if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) {
    +            log.warn(s"Executor with id $executorId was not marked as disconnected, but the" +
    +              s" watch received an event of type $action for this executor. The executor may" +
    +              " have failed to start in the first place and never registered with the driver.")
    +          }
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +
    +        case _ => logDebug(s"Received event of executor pod $podName: " + action)
    +      }
    +    }
    +
    +    override def onClose(cause: KubernetesClientException): Unit = {
    +      logDebug("Executor pod watch closed.", cause)
    +    }
    +
    +    private def getExecutorExitStatus(pod: Pod): Int = {
    +      val containerStatuses = pod.getStatus.getContainerStatuses
    +      if (!containerStatuses.isEmpty) {
    +        // we assume the first container represents the pod status. This assumption may not hold
    +        // true in the future. Revisit this if side-car containers start running inside executor
    +        // pods.
    +        getExecutorExitStatus(containerStatuses.get(0))
    +      } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS
    +    }
    +
    +    private def getExecutorExitStatus(containerStatus: ContainerStatus): Int = {
    +      Option(containerStatus.getState).map { containerState =>
    +        Option(containerState.getTerminated).map { containerStateTerminated =>
    +          containerStateTerminated.getExitCode.intValue()
    +        }.getOrElse(UNKNOWN_EXIT_CODE)
    +      }.getOrElse(UNKNOWN_EXIT_CODE)
    +    }
    +
    +    private def isPodAlreadyReleased(pod: Pod): Boolean = {
    +      val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL)
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        !runningExecutorsToPods.contains(executorId)
    +      }
    +    }
    +
    +    private def executorExitReasonOnError(pod: Pod): ExecutorExited = {
    +      val containerExitStatus = getExecutorExitStatus(pod)
    +      // container was probably actively killed by the driver.
    +      if (isPodAlreadyReleased(pod)) {
    +        ExecutorExited(containerExitStatus, exitCausedByApp = false,
    +          s"Container in pod ${pod.getMetadata.getName} exited from explicit termination" +
    +            " request.")
    +      } else {
    +        val containerExitReason = s"Pod ${pod.getMetadata.getName}'s executor container " +
    +          s"exited with exit status code $containerExitStatus."
    +        ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason)
    +      }
    +    }
    +
    +    private def executorExitReasonOnDelete(pod: Pod): ExecutorExited = {
    +      val exitMessage = if (isPodAlreadyReleased(pod)) {
    +        s"Container in pod ${pod.getMetadata.getName} exited from explicit termination request."
    +      } else {
    +        s"Pod ${pod.getMetadata.getName} deleted or lost."
    +      }
    +      ExecutorExited(getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage)
    +    }
    +
    +    private def getExecutorId(pod: Pod): String = {
    +      val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL)
    +      require(executorId != null, "Unexpected pod metadata; expected all executor pods " +
    +        s"to have label $SPARK_EXECUTOR_ID_LABEL.")
    +      executorId
    +    }
    +  }
    +
    +  override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = {
    +    new KubernetesDriverEndpoint(rpcEnv, properties)
    +  }
    +
    +  private class KubernetesDriverEndpoint(
    +      rpcEnv: RpcEnv,
    +      sparkProperties: Seq[(String, String)])
    +    extends DriverEndpoint(rpcEnv, sparkProperties) {
    +
    +    override def onDisconnected(rpcAddress: RpcAddress): Unit = {
    +      addressToExecutorId.get(rpcAddress).foreach { executorId =>
    +        if (disableExecutor(executorId)) {
    +          RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +            runningExecutorsToPods.get(executorId).foreach { pod =>
    +              disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            }
    +          }
    +        }
    +      }
    +    }
    +  }
    +}
    +
    +private object KubernetesClusterSchedulerBackend {
    +  private val UNKNOWN_EXIT_CODE = -1
    +  // Number of times we are allowed check for the loss reason for an executor before we give up
    +  // and assume the executor failed for good, and attribute it to a framework fault.
    +  val MAX_EXECUTOR_LOST_REASON_CHECKS = 10
    --- End diff --
    
    Actually I think we can make it a config property with a sensible default value, given that the batch delay is already configurable. 


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #84096 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84096/testReport)** for PR 19468 at commit [`3b587b4`](https://github.com/apache/spark/commit/3b587b4b4362f184b148c22522821ef7b163717e).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r148916475
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,227 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    --- End diff --
    
    Done


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153084513
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SPARK_VERSION
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +private[spark] object Config extends Logging {
    +
    +  val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$SPARK_VERSION")
    +
    +  val DOCKER_IMAGE_PULL_POLICY =
    +    ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
    +      .doc("Kubernetes image pull policy. Valid values are Always, Never, and IfNotPresent.")
    +      .stringConf
    +      .checkValues(Set("Always", "Never", "IfNotPresent"))
    +      .createWithDefault("IfNotPresent")
    +
    +  val APISERVER_AUTH_DRIVER_CONF_PREFIX =
    +      "spark.kubernetes.authenticate.driver"
    +  val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
    +      "spark.kubernetes.authenticate.driver.mounted"
    +  val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken"
    +  val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile"
    +  val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile"
    +  val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile"
    +  val CA_CERT_FILE_CONF_SUFFIX = "caCertFile"
    +
    +  val KUBERNETES_SERVICE_ACCOUNT_NAME =
    +    ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName")
    +      .doc("Service account that is used when running the driver pod. The driver pod uses" +
    +        " this service account when requesting executor pods from the API server. If specific" +
    +        " credentials are given for the driver pod to use, the driver will favor" +
    +        " using those credentials instead.")
    +      .stringConf
    +      .createOptional
    +
    +  // Note that while we set a default for this when we start up the
    +  // scheduler, the specific default value is dynamically determined
    +  // based on the executor memory.
    +  val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD =
    +    ConfigBuilder("spark.kubernetes.executor.memoryOverhead")
    +      .doc("The amount of off-heap memory (in megabytes) to be allocated per executor. This" +
    +        " is memory that accounts for things like VM overheads, interned strings, other native" +
    +        " overheads, etc. This tends to grow with the executor size. (typically 6-10%).")
    +      .bytesConf(ByteUnit.MiB)
    +      .createOptional
    --- End diff --
    
    What about driver memory overhead ?
    I see that mesos does not support it, while yarn does - is it relevant here ?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150741895
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala ---
    @@ -0,0 +1,114 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.{SPARK_VERSION => sparkVersion}
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +package object config extends Logging {
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150959150
  
    --- 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,
    --- End diff --
    
    If this is purely for preventing future changes from potentially breaking it, why cannot we verify that the contract is held in integration tests so the integration tests will fail if the contract is indeed broken?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r143916064
  
    --- Diff: pom.xml ---
    @@ -2649,6 +2649,13 @@
         </profile>
     
         <profile>
    +      <id>kubernetes</id>
    --- End diff --
    
    this new profile is not use in build script, shall we add this so that the new contributions in this PR will be built and tested?


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83789 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83789/testReport)** for PR 19468 at commit [`1f271be`](https://github.com/apache/spark/commit/1f271bee5b43c03fab6d2c55d96a93f417cc6915).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83903 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83903/testReport)** for PR 19468 at commit [`7f14b71`](https://github.com/apache/spark/commit/7f14b71c0254553893ac70642d0f0b77bb7687a2).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150754184
  
    --- 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,
    --- End diff --
    
    +1 to @liyinan926's comment. Since we're directly getting the pod from `ExecutorPodFactory`, the guard is superfluous and basically dead code. @mccheah PTAL.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146426271
  
    --- 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 --
    
    Is there anywhere where these spark-submit changes you mention can be seen? You're starting to get me worried about what you guys are doing.
    
    spark-submit is supposed to be dumb. It just runs a class; in client mode that's the user's main class, which should eventually create a `SparkContext` and trigger this code.
    
    In cluster mode it's a cluster-manager specific class that submits the user application to run in the cluster somewhere.
    
    I can't see why you'd need a pre-computed app id in either case unless you're trying to make spark-submit do much more than it should.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83100 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83100/testReport)** for PR 19468 at commit [`2b5dcac`](https://github.com/apache/spark/commit/2b5dcac4f188b8c61b67aec131a1eb7d91968356).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147017011
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,440 @@
    +/*
    + * 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.JavaConverters._
    +import scala.collection.mutable
    +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 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 = 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)
    --- End diff --
    
    But if we're not losing anything correctness-wise by moving the potentially slow calls out of synchronized blocks then we should do so. I think it's safe to move the code out, but will double check and mention if there are problems with that.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #82938 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82938/testReport)** for PR 19468 at commit [`c052212`](https://github.com/apache/spark/commit/c052212888e01eac90a006bfb5d14c513e33d0a3).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83830 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83830/testReport)** for PR 19468 at commit [`df03462`](https://github.com/apache/spark/commit/df03462096e9f21f11241078ae120dd0cfb9f226).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147014681
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    --- End diff --
    
    getDouble will always return double.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83062 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83062/testReport)** for PR 19468 at commit [`a4f9797`](https://github.com/apache/spark/commit/a4f97976afff452d7d953b83b722da61dfb40c3b).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83851 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83851/testReport)** for PR 19468 at commit [`0ab9ca7`](https://github.com/apache/spark/commit/0ab9ca7b2056c0de7a820fa7bb9391227bcf5275).
     * This patch **fails RAT tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by markhamstra <gi...@git.apache.org>.
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-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83851/
    Test FAILed.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    ping @jiangxb1987 


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r149172795
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,425 @@
    +/*
    + * 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)
    +  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()
    +      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),
    --- End diff --
    
    This lgtm as is. Getting the info from the pod itself makes sense. 


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153272632
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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.deploy.k8s
    +
    +private[spark] object Constants {
    +
    +  // Labels
    +  val SPARK_APP_ID_LABEL = "spark-app-selector"
    +  val SPARK_EXECUTOR_ID_LABEL = "spark-exec-id"
    +  val SPARK_ROLE_LABEL = "spark-role"
    +  val SPARK_POD_DRIVER_ROLE = "driver"
    +  val SPARK_POD_EXECUTOR_ROLE = "executor"
    +
    +  // Default and fixed ports
    +  val DEFAULT_DRIVER_PORT = 7078
    +  val DEFAULT_BLOCKMANAGER_PORT = 7079
    +  val BLOCK_MANAGER_PORT_NAME = "blockmanager"
    +  val EXECUTOR_PORT_NAME = "executor"
    +
    +  // Environment Variables
    +  val ENV_EXECUTOR_PORT = "SPARK_EXECUTOR_PORT"
    +  val ENV_DRIVER_URL = "SPARK_DRIVER_URL"
    +  val ENV_EXECUTOR_CORES = "SPARK_EXECUTOR_CORES"
    +  val ENV_EXECUTOR_MEMORY = "SPARK_EXECUTOR_MEMORY"
    +  val ENV_APPLICATION_ID = "SPARK_APPLICATION_ID"
    +  val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID"
    +  val ENV_EXECUTOR_POD_IP = "SPARK_EXECUTOR_POD_IP"
    +  val ENV_EXECUTOR_EXTRA_CLASSPATH = "SPARK_EXECUTOR_EXTRA_CLASSPATH"
    +  val ENV_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH"
    +  val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_"
    --- End diff --
    
    What do you mean by shared?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150743119
  
    --- 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 {
    +      for (executor <- executorIds) {
    +        val maybeRemovedExecutor = runningExecutorsToPods.remove(executor)
    +        maybeRemovedExecutor.foreach { executorPod =>
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod)
    +          podsToDelete += executorPod
    +        }
    +        if (maybeRemovedExecutor.isEmpty) {
    +          logWarning(s"Unable to remove pod for unknown executor $executor")
    +        }
    +      }
    +    }
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +          && pod.getMetadata.getDeletionTimestamp == null) =>
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152445984
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    --- End diff --
    
    We bias towards keeping Spark configuration consistent across resource managers and then translating it to what Kubernetes expects. Should we instead be accepting Kubernetes-formatted memory strings and translating it on the environment variable?
    
    Regardless, there is going to be a difference in the string we send to the JVM in `-Xmx` and `-Xms` and what we pass to Kubernetes. We could translate everything to just raw bytes to keep everything consistent.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    I went through the changes to make sure the non-k8s changes are ok. They do look ok to me. From that perspective, LGTM.



---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83061 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83061/testReport)** for PR 19468 at commit [`b0a5839`](https://github.com/apache/spark/commit/b0a5839684d3eb08e1ad93db735440c26b3a1d1e).


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145249881
  
    --- Diff: dev/sparktestsupport/modules.py ---
    @@ -531,6 +531,14 @@ def __hash__(self):
         sbt_test_goals=["mesos/test"]
     )
     
    +kubernetes = Module(
    +    name="kubernetes",
    +    dependencies=[],
    +    source_file_regexes=["resource-managers/kubernetes/core"],
    --- End diff --
    
    remove 'core'


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147001175
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    --- End diff --
    
    Cores can be a double in the Kubernetes context because fractional cores are allowed in Kubernetes.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150748702
  
    --- 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,
    --- End diff --
    
    I don't really see a possibility for a successfully constructed executor pod to not have this label with the right value. There's already checks in `ExecutorPodFactory` that guard against users setting this label. So I don't think the checks here are really necessary. @foxish to confirm.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147017517
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId),
    +      (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    --- End diff --
    
    Did a quick scan - looks like we use that call in many places. But given our heavy reliance on hostnames, how would this manifest in practice? Is it the case where Spark shouldn't be working at all? We haven't seen any problems specifically with the hostnames of executors, and we recently fixed needing a hostname for the driver by putting the driver behind a service.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83800 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83800/testReport)** for PR 19468 at commit [`71a971f`](https://github.com/apache/spark/commit/71a971f2108fd9e04532fe73ecbd1ec00b36d132).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145260156
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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.{ExecutorService, ScheduledExecutorService, TimeUnit}
    +
    +import scala.collection.JavaConverters._
    +import scala.concurrent.Future
    +
    +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder, PodList}
    +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher}
    +import io.fabric8.kubernetes.client.Watcher.Action
    +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NonNamespaceOperation, PodResource}
    +import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations}
    +import org.mockito.Matchers.{any, eq => mockitoEq}
    +import org.mockito.Mockito.{mock => _, _}
    +import org.scalatest.BeforeAndAfter
    +import org.scalatest.mock.MockitoSugar._
    +
    +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.rpc._
    +import org.apache.spark.scheduler.{ExecutorExited, LiveListenerBus, SlaveLost, TaskSchedulerImpl}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +private[spark] class KubernetesClusterSchedulerBackendSuite
    +    extends SparkFunSuite with BeforeAndAfter {
    +
    +  private val APP_ID = "test-spark-app"
    +  private val DRIVER_POD_NAME = "spark-driver-pod"
    +  private val NAMESPACE = "test-namespace"
    +  private val SPARK_DRIVER_HOST = "localhost"
    +  private val SPARK_DRIVER_PORT = 7077
    +  private val POD_ALLOCATION_INTERVAL = 60L
    +  private val DRIVER_URL = RpcEndpointAddress(
    +      SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
    +  private val FIRST_EXECUTOR_POD = new PodBuilder()
    +    .withNewMetadata()
    +      .withName("pod1")
    +      .endMetadata()
    +    .withNewSpec()
    +      .withNodeName("node1")
    +      .endSpec()
    +    .withNewStatus()
    +      .withHostIP("192.168.99.100")
    +      .endStatus()
    +    .build()
    +  private val SECOND_EXECUTOR_POD = new PodBuilder()
    +    .withNewMetadata()
    +      .withName("pod2")
    +      .endMetadata()
    +    .withNewSpec()
    +      .withNodeName("node2")
    +      .endSpec()
    +    .withNewStatus()
    +      .withHostIP("192.168.99.101")
    +      .endStatus()
    +    .build()
    +
    +  private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]
    +  private type LABELLED_PODS = FilterWatchListDeletable[
    --- End diff --
    
    labeled


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153332360
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,226 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX)
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX)
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX)
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1)
    +  private val executorLimitCores = sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId)) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    +    ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq
    +    val requiredPorts = Seq(
    +      (EXECUTOR_PORT_NAME, executorPort),
    +      (BLOCK_MANAGER_PORT_NAME, blockManagerPort))
    +      .map { case (name, port) =>
    +        new ContainerPortBuilder()
    +          .withName(name)
    +          .withContainerPort(port)
    +          .build()
    +      }
    +
    +    val executorContainer = new ContainerBuilder()
    +      .withName("executor")
    +      .withImage(executorDockerImage)
    +      .withImagePullPolicy(dockerImagePullPolicy)
    +      .withNewResources()
    +        .addToRequests("memory", executorMemoryQuantity)
    +        .addToLimits("memory", executorMemoryLimitQuantity)
    +        .addToRequests("cpu", executorCpuQuantity)
    +        .endResources()
    +      .addAllToEnv(executorEnv.asJava)
    +      .withPorts(requiredPorts.asJava)
    +      .build()
    +
    +    val executorPod = new PodBuilder()
    +      .withNewMetadata()
    +        .withName(name)
    +        .withLabels(resolvedExecutorLabels.asJava)
    +        .withAnnotations(executorAnnotations.asJava)
    +        .withOwnerReferences()
    +          .addNewOwnerReference()
    +            .withController(true)
    +            .withApiVersion(driverPod.getApiVersion)
    +            .withKind(driverPod.getKind)
    +            .withName(driverPod.getMetadata.getName)
    +            .withUid(driverPod.getMetadata.getUid)
    +            .endOwnerReference()
    +        .endMetadata()
    +      .withNewSpec()
    +        .withHostname(hostname)
    +        .withRestartPolicy("Never")
    +        .withNodeSelector(nodeSelector.asJava)
    +        .endSpec()
    +      .build()
    +
    +    val containerWithExecutorLimitCores = executorLimitCores.map { limitCores =>
    +      val executorCpuLimitQuantity = new QuantityBuilder(false)
    +        .withAmount(limitCores)
    +        .build()
    +      new ContainerBuilder(executorContainer)
    +        .editResources()
    +        .addToLimits("cpu", executorCpuLimitQuantity)
    +        .endResources()
    +        .build()
    +    }.getOrElse(executorContainer)
    +
    +    new PodBuilder(executorPod)
    +      .editSpec()
    +        .addToContainers(containerWithExecutorLimitCores)
    +        .endSpec()
    +      .build()
    +  }
    --- End diff --
    
    It's somewhat a limitation of the current integration. Allowing users to use customized log4j configuration needs the ability to inject user-specified configuration files like `log4j.properties` through ConfigMaps into the driver and executor pods. This is not yet supported but will likely be supported in the near future. On the other hand, it is recommended in Kubernetes to log to stdout/stderr so logs are handled by kubelets and can be retrieved using `kubectl logs`.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by foxish <gi...@git.apache.org>.
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83057 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83057/testReport)** for PR 19468 at commit [`2fb596d`](https://github.com/apache/spark/commit/2fb596d024b4115c9209df6e9fae24550e08a89b).


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Thanks - merging in master!



---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147022826
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId),
    +      (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    --- End diff --
    
    To add, if the list above is not accurate (needs additions/mods/deletions), that is fine - I was listing potential concerns from top of my head.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84234/
    Test PASSed.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83425 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83425/testReport)** for PR 19468 at commit [`6cf4ed7`](https://github.com/apache/spark/commit/6cf4ed7eec3f8a1798d260622ab5641db92ab13d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153351876
  
    --- 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 =>
    --- End diff --
    
    Never mind, am looking at an older commit.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150495130
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala ---
    @@ -0,0 +1,113 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.{SPARK_VERSION => sparkVersion}
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +package object config extends Logging {
    +
    +  private[spark] val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  private[spark] val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$sparkVersion")
    +
    +  private[spark] val DOCKER_IMAGE_PULL_POLICY =
    +    ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
    +      .doc("Docker image pull policy when pulling any docker image in Kubernetes integration")
    +      .stringConf
    +      .createWithDefault("IfNotPresent")
    +
    --- End diff --
    
    nit: duplicated empty lines.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150743124
  
    --- 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 {
    +      for (executor <- executorIds) {
    +        val maybeRemovedExecutor = runningExecutorsToPods.remove(executor)
    +        maybeRemovedExecutor.foreach { executorPod =>
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod)
    +          podsToDelete += executorPod
    +        }
    +        if (maybeRemovedExecutor.isEmpty) {
    +          logWarning(s"Unable to remove pod for unknown executor $executor")
    +        }
    +      }
    +    }
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +          && pod.getMetadata.getDeletionTimestamp == null) =>
    +          val podIP = pod.getStatus.getPodIP
    +          val clusterNodeName = pod.getSpec.getNodeName
    +          logInfo(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.")
    +          executorPodsByIPs.put(podIP, pod)
    +        case Action.DELETED | Action.ERROR =>
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146940612
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.{SPARK_VERSION => sparkVersion}
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +package object config extends Logging {
    +
    +  private[spark] val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  private[spark] val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$sparkVersion")
    +
    +  private[spark] val DOCKER_IMAGE_PULL_POLICY =
    +    ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
    +      .doc("Docker image pull policy when pulling any docker image in Kubernetes integration")
    +      .stringConf
    +      .createWithDefault("IfNotPresent")
    +
    +
    +  private[spark] val APISERVER_AUTH_DRIVER_CONF_PREFIX =
    +      "spark.kubernetes.authenticate.driver"
    +  private[spark] val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
    +      "spark.kubernetes.authenticate.driver.mounted"
    +  private[spark] val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken"
    +  private[spark] val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile"
    +  private[spark] val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile"
    +  private[spark] val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile"
    +  private[spark] val CA_CERT_FILE_CONF_SUFFIX = "caCertFile"
    +
    +  private[spark] val KUBERNETES_SERVICE_ACCOUNT_NAME =
    +    ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName")
    +      .doc("Service account that is used when running the driver pod. The driver pod uses" +
    +        " this service account when requesting executor pods from the API server. If specific" +
    +        " credentials are given for the driver pod to use, the driver will favor" +
    +        " using those credentials instead.")
    +      .stringConf
    +      .createOptional
    +
    +  // Note that while we set a default for this when we start up the
    +  // scheduler, the specific default value is dynamically determined
    +  // based on the executor memory.
    +  private[spark] val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD =
    +    ConfigBuilder("spark.kubernetes.executor.memoryOverhead")
    +      .doc("The amount of off-heap memory (in megabytes) to be allocated per executor. This" +
    +        " is memory that accounts for things like VM overheads, interned strings, other native" +
    +        " overheads, etc. This tends to grow with the executor size. (typically 6-10%).")
    +      .bytesConf(ByteUnit.MiB)
    +      .createOptional
    +
    +  private[spark] val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label."
    +  private[spark] val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation."
    +
    +  private[spark] val KUBERNETES_DRIVER_POD_NAME =
    +    ConfigBuilder("spark.kubernetes.driver.pod.name")
    +      .doc("Name of the driver pod.")
    +      .stringConf
    +      .createOptional
    +
    +  private[spark] val KUBERNETES_EXECUTOR_POD_NAME_PREFIX =
    +    ConfigBuilder("spark.kubernetes.executor.podNamePrefix")
    +      .doc("Prefix to use in front of the executor pod names.")
    +      .internal()
    +      .stringConf
    +      .createWithDefault("spark")
    +
    +  private[spark] val KUBERNETES_ALLOCATION_BATCH_SIZE =
    +    ConfigBuilder("spark.kubernetes.allocation.batch.size")
    +      .doc("Number of pods to launch at once in each round of dynamic allocation. ")
    --- End diff --
    
    nit: extra space at the end of string here and L102


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82854/
    Test FAILed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153356659
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,226 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX)
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX)
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX)
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1)
    +  private val executorLimitCores = sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    --- End diff --
    
    I am fine with deferring this in a later PR as long as we track it somewhere : it might so happen that this is a requirement anyway and we cant use random names.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145279283
  
    --- 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())
    +
    +  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
    +    // TODO investigate why Utils.tryLogNonFatalError() doesn't work in this context.
    +    // When using Utils.tryLogNonFatalError some of the code fails but without any logs or
    +    // indication as to why.
    +    try {
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        runningExecutorsToPods.values.foreach(kubernetesClient.pods().delete(_))
    +        runningExecutorsToPods.clear()
    +        runningPodsToExecutors.clear()
    +      }
    +      executorPodsByIPs.clear()
    +      val resource = executorWatchResource.getAndSet(null)
    +      if (resource != null) {
    +        resource.close()
    +      }
    +    } catch {
    +      case e: Throwable => logError("Uncaught exception while shutting down controllers.", e)
    +    }
    +    try {
    +      logInfo("Closing kubernetes client")
    +      kubernetesClient.close()
    +    } catch {
    +      case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e)
    +    }
    +  }
    +
    +  /**
    +   * @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]() ++
    +      KubernetesClusterSchedulerBackend.this.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]
    +  }
    +
    +  /**
    +   * Allocates a new executor pod
    +   *
    +   * @param nodeToLocalTaskCount  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.
    +   * @return A tuple of the new executor name and the Pod data structure.
    +   */
    +  private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = {
    +    val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString
    +    val executorPod = executorPodFactory.createExecutorPod(
    +        executorId,
    +        applicationId(),
    +        driverUrl,
    +        conf.getExecutorEnv,
    +        driverPod,
    +        nodeToLocalTaskCount)
    +    try {
    +      (executorId, kubernetesClient.pods.create(executorPod))
    +    } catch {
    +      case throwable: Throwable =>
    +        logError("Failed to allocate executor pod.", throwable)
    +        throw throwable
    +    }
    +  }
    +
    +  override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] {
    +    totalExpectedExecutors.set(requestedTotal)
    +    true
    +  }
    +
    +  override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] {
    +    RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +      for (executor <- executorIds) {
    +        val maybeRemovedExecutor = runningExecutorsToPods.remove(executor)
    +        maybeRemovedExecutor.foreach { executorPod =>
    +          kubernetesClient.pods().delete(executorPod)
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod)
    +          runningPodsToExecutors.remove(executorPod.getMetadata.getName)
    +        }
    +        if (maybeRemovedExecutor.isEmpty) {
    +          logWarning(s"Unable to remove pod for unknown executor $executor")
    +        }
    +      }
    +    }
    +    true
    +  }
    +
    +  def getExecutorPodByIP(podIP: String): Option[Pod] = {
    +    // Note: Per https://github.com/databricks/scala-style-guide#concurrency, we don't
    +    // want to be switching to scala.collection.concurrent.Map on
    +    // executorPodsByIPs.
    +    val pod = executorPodsByIPs.get(podIP)
    +    Option(pod)
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      if (action == Action.MODIFIED && pod.getStatus.getPhase == "Running"
    +        && pod.getMetadata.getDeletionTimestamp == null) {
    +        val podIP = pod.getStatus.getPodIP
    +        val clusterNodeName = pod.getSpec.getNodeName
    +        logDebug(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.")
    +        executorPodsByIPs.put(podIP, pod)
    +      } else if ((action == Action.MODIFIED && pod.getMetadata.getDeletionTimestamp != null) ||
    +        action == Action.DELETED || action == Action.ERROR) {
    +        val podName = pod.getMetadata.getName
    +        val podIP = pod.getStatus.getPodIP
    +        logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +        if (podIP != null) {
    +          executorPodsByIPs.remove(podIP)
    +        }
    +        if (action == Action.ERROR) {
    +          logInfo(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason)
    +          handleErroredPod(pod)
    +        } else if (action == Action.DELETED) {
    +          logInfo(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason)
    +          handleDeletedPod(pod)
    +        }
    +      }
    +    }
    +
    +    override def onClose(cause: KubernetesClientException): Unit = {
    +      logDebug("Executor pod watch closed.", cause)
    +    }
    +
    +    def getExecutorExitStatus(pod: Pod): Int = {
    +      val containerStatuses = pod.getStatus.getContainerStatuses
    +      if (!containerStatuses.isEmpty) {
    +        // we assume the first container represents the pod status. This assumption may not hold
    +        // true in the future. Revisit this if side-car containers start running inside executor
    +        // pods.
    +        getExecutorExitStatus(containerStatuses.get(0))
    +      } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS
    +    }
    +
    +    def getExecutorExitStatus(containerStatus: ContainerStatus): Int = {
    +      Option(containerStatus.getState).map(containerState =>
    +        Option(containerState.getTerminated).map(containerStateTerminated =>
    +          containerStateTerminated.getExitCode.intValue()).getOrElse(UNKNOWN_EXIT_CODE)
    +      ).getOrElse(UNKNOWN_EXIT_CODE)
    +    }
    +
    +    def isPodAlreadyReleased(pod: Pod): Boolean = {
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        !runningPodsToExecutors.contains(pod.getMetadata.getName)
    +      }
    +    }
    +
    +    def handleErroredPod(pod: Pod): Unit = {
    +      val containerExitStatus = getExecutorExitStatus(pod)
    +      // container was probably actively killed by the driver.
    +      val exitReason = if (isPodAlreadyReleased(pod)) {
    +        ExecutorExited(containerExitStatus, exitCausedByApp = false,
    +          s"Container in pod " + pod.getMetadata.getName +
    +            " exited from explicit termination request.")
    +      } else {
    +        val containerExitReason = containerExitStatus match {
    +          case VMEM_EXCEEDED_EXIT_CODE | PMEM_EXCEEDED_EXIT_CODE =>
    +            memLimitExceededLogMessage(pod.getStatus.getReason)
    +          case _ =>
    +            // Here we can't be sure that that exit was caused by the application but this seems
    +            // to be the right default since we know the pod was not explicitly deleted by
    +            // the user.
    +            s"Pod ${pod.getMetadata.getName}'s executor container exited with exit status" +
    +              s" code $containerExitStatus."
    +        }
    +        ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason)
    +      }
    +      podsWithKnownExitReasons.put(pod.getMetadata.getName, exitReason)
    +    }
    +
    +    def handleDeletedPod(pod: Pod): Unit = {
    +      val exitMessage = if (isPodAlreadyReleased(pod)) {
    +        s"Container in pod ${pod.getMetadata.getName} exited from explicit termination request."
    +      } else {
    +        s"Pod ${pod.getMetadata.getName} deleted or lost."
    +      }
    +      val exitReason = ExecutorExited(
    +        getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage)
    +      podsWithKnownExitReasons.put(pod.getMetadata.getName, exitReason)
    +    }
    +  }
    +
    +  override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = {
    +    new KubernetesDriverEndpoint(rpcEnv, properties)
    +  }
    +
    +  private class KubernetesDriverEndpoint(
    +    rpcEnv: RpcEnv,
    +    sparkProperties: Seq[(String, String)])
    +    extends DriverEndpoint(rpcEnv, sparkProperties) {
    +
    +    override def onDisconnected(rpcAddress: RpcAddress): Unit = {
    +      addressToExecutorId.get(rpcAddress).foreach { executorId =>
    +        if (disableExecutor(executorId)) {
    +          RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +            runningExecutorsToPods.get(executorId).foreach { pod =>
    +              disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            }
    +          }
    +        }
    +      }
    +    }
    +  }
    +}
    +
    +private object KubernetesClusterSchedulerBackend {
    +  private val VMEM_EXCEEDED_EXIT_CODE = -103
    --- End diff --
    
    @varunkatta PTAL


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150545913
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,425 @@
    +/*
    + * 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)
    +  require(podAllocationInterval > 0, "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, "Allocation batch size " +
    --- End diff --
    
    ditto


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83057 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83057/testReport)** for PR 19468 at commit [`2fb596d`](https://github.com/apache/spark/commit/2fb596d024b4115c9209df6e9fae24550e08a89b).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83062 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83062/testReport)** for PR 19468 at commit [`a4f9797`](https://github.com/apache/spark/commit/a4f97976afff452d7d953b83b722da61dfb40c3b).


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84228/
    Test FAILed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146973841
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    --- End diff --
    
    nit: 1d -> 1 ?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145260500
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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.{ExecutorService, ScheduledExecutorService, TimeUnit}
    +
    +import scala.collection.JavaConverters._
    +import scala.concurrent.Future
    +
    +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder, PodList}
    +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher}
    +import io.fabric8.kubernetes.client.Watcher.Action
    +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NonNamespaceOperation, PodResource}
    +import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations}
    +import org.mockito.Matchers.{any, eq => mockitoEq}
    +import org.mockito.Mockito.{mock => _, _}
    +import org.scalatest.BeforeAndAfter
    +import org.scalatest.mock.MockitoSugar._
    +
    +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.rpc._
    +import org.apache.spark.scheduler.{ExecutorExited, LiveListenerBus, SlaveLost, TaskSchedulerImpl}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +private[spark] class KubernetesClusterSchedulerBackendSuite
    +    extends SparkFunSuite with BeforeAndAfter {
    +
    +  private val APP_ID = "test-spark-app"
    +  private val DRIVER_POD_NAME = "spark-driver-pod"
    +  private val NAMESPACE = "test-namespace"
    +  private val SPARK_DRIVER_HOST = "localhost"
    +  private val SPARK_DRIVER_PORT = 7077
    +  private val POD_ALLOCATION_INTERVAL = 60L
    +  private val DRIVER_URL = RpcEndpointAddress(
    +      SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
    +  private val FIRST_EXECUTOR_POD = new PodBuilder()
    +    .withNewMetadata()
    +      .withName("pod1")
    +      .endMetadata()
    +    .withNewSpec()
    +      .withNodeName("node1")
    +      .endSpec()
    +    .withNewStatus()
    +      .withHostIP("192.168.99.100")
    +      .endStatus()
    +    .build()
    +  private val SECOND_EXECUTOR_POD = new PodBuilder()
    +    .withNewMetadata()
    +      .withName("pod2")
    +      .endMetadata()
    +    .withNewSpec()
    +      .withNodeName("node2")
    +      .endSpec()
    +    .withNewStatus()
    +      .withHostIP("192.168.99.101")
    +      .endStatus()
    +    .build()
    +
    +  private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]
    +  private type LABELLED_PODS = FilterWatchListDeletable[
    +      Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]]
    +  private type IN_NAMESPACE_PODS = NonNamespaceOperation[
    +      Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]
    +
    +  @Mock
    +  private var sparkContext: SparkContext = _
    +
    +  @Mock
    +  private var listenerBus: LiveListenerBus = _
    +
    +  @Mock
    +  private var taskSchedulerImpl: TaskSchedulerImpl = _
    +
    +  @Mock
    +  private var allocatorExecutor: ScheduledExecutorService = _
    +
    +  @Mock
    +  private var requestExecutorsService: ExecutorService = _
    +
    +  @Mock
    +  private var executorPodFactory: ExecutorPodFactory = _
    +
    +  @Mock
    +  private var kubernetesClient: KubernetesClient = _
    +
    +  @Mock
    +  private var podOperations: PODS = _
    +
    +  @Mock
    +  private var podsWithLabelOperations: LABELLED_PODS = _
    +
    +  @Mock
    +  private var podsInNamespace: IN_NAMESPACE_PODS = _
    +
    +  @Mock
    +  private var podsWithDriverName: PodResource[Pod, DoneablePod] = _
    +
    +  @Mock
    +  private var rpcEnv: RpcEnv = _
    +
    +  @Mock
    +  private var driverEndpointRef: RpcEndpointRef = _
    +
    +  @Mock
    +  private var executorPodsWatch: Watch = _
    +
    +  private var sparkConf: SparkConf = _
    +  private var executorPodsWatcherArgument: ArgumentCaptor[Watcher[Pod]] = _
    +  private var allocatorRunnable: ArgumentCaptor[Runnable] = _
    +  private var requestExecutorRunnable: ArgumentCaptor[Runnable] = _
    +  private var driverEndpoint: ArgumentCaptor[RpcEndpoint] = _
    +
    +  private val driverPod = new PodBuilder()
    +    .withNewMetadata()
    +      .withName(DRIVER_POD_NAME)
    +      .addToLabels(SPARK_APP_ID_LABEL, APP_ID)
    +      .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_DRIVER_ROLE)
    +      .endMetadata()
    +    .build()
    +
    +  before {
    +    MockitoAnnotations.initMocks(this)
    +    sparkConf = new SparkConf()
    +        .set("spark.app.id", APP_ID)
    +        .set(KUBERNETES_DRIVER_POD_NAME, DRIVER_POD_NAME)
    +        .set(KUBERNETES_NAMESPACE, NAMESPACE)
    +        .set("spark.driver.host", SPARK_DRIVER_HOST)
    +        .set("spark.driver.port", SPARK_DRIVER_PORT.toString)
    +        .set(KUBERNETES_ALLOCATION_BATCH_DELAY, POD_ALLOCATION_INTERVAL)
    +    executorPodsWatcherArgument = ArgumentCaptor.forClass(classOf[Watcher[Pod]])
    +    allocatorRunnable = ArgumentCaptor.forClass(classOf[Runnable])
    +    requestExecutorRunnable = ArgumentCaptor.forClass(classOf[Runnable])
    +    driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint])
    +    when(sparkContext.conf).thenReturn(sparkConf)
    +    when(sparkContext.listenerBus).thenReturn(listenerBus)
    +    when(taskSchedulerImpl.sc).thenReturn(sparkContext)
    +    when(kubernetesClient.pods()).thenReturn(podOperations)
    +    when(podOperations.withLabel(SPARK_APP_ID_LABEL, APP_ID)).thenReturn(podsWithLabelOperations)
    +    when(podsWithLabelOperations.watch(executorPodsWatcherArgument.capture()))
    +        .thenReturn(executorPodsWatch)
    +    when(podOperations.inNamespace(NAMESPACE)).thenReturn(podsInNamespace)
    +    when(podsInNamespace.withName(DRIVER_POD_NAME)).thenReturn(podsWithDriverName)
    +    when(podsWithDriverName.get()).thenReturn(driverPod)
    +    when(allocatorExecutor.scheduleWithFixedDelay(
    +        allocatorRunnable.capture(),
    +        mockitoEq(0L),
    +        mockitoEq(POD_ALLOCATION_INTERVAL),
    +        mockitoEq(TimeUnit.SECONDS))).thenReturn(null)
    +    // Creating Futures in Scala backed by a Java executor service resolves to running
    +    // ExecutorService#execute (as opposed to submit)
    +    doNothing().when(requestExecutorsService).execute(requestExecutorRunnable.capture())
    +    when(rpcEnv.setupEndpoint(
    +        mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture()))
    +        .thenReturn(driverEndpointRef)
    +    when(driverEndpointRef.ask[Boolean]
    +      (any(classOf[Any]))
    +      (any())).thenReturn(mock[Future[Boolean]])
    +  }
    +
    +  test("Basic lifecycle expectations when starting and stopping the scheduler.") {
    +    val scheduler = newSchedulerBackend()
    +    scheduler.start()
    +    assert(executorPodsWatcherArgument.getValue != null)
    +    assert(allocatorRunnable.getValue != null)
    +    scheduler.stop()
    +    verify(executorPodsWatch).close()
    +  }
    +
    +  test("Static allocation should request executors upon first allocator run.") {
    +    sparkConf
    +        .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2)
    +        .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2)
    +    val scheduler = newSchedulerBackend()
    +    scheduler.start()
    +    requestExecutorRunnable.getValue.run()
    +    expectPodCreationWithId(1, FIRST_EXECUTOR_POD)
    +    expectPodCreationWithId(2, SECOND_EXECUTOR_POD)
    +    when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg())
    +    allocatorRunnable.getValue.run()
    +    verify(podOperations).create(FIRST_EXECUTOR_POD)
    +    verify(podOperations).create(SECOND_EXECUTOR_POD)
    +  }
    +
    +  test("Killing executors deletes the executor pods") {
    +    sparkConf
    +        .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2)
    +        .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2)
    +    val scheduler = newSchedulerBackend()
    +    scheduler.start()
    +    requestExecutorRunnable.getValue.run()
    +    expectPodCreationWithId(1, FIRST_EXECUTOR_POD)
    +    expectPodCreationWithId(2, SECOND_EXECUTOR_POD)
    +    when(podOperations.create(any(classOf[Pod])))
    +        .thenAnswer(AdditionalAnswers.returnsFirstArg())
    +    allocatorRunnable.getValue.run()
    +    scheduler.doKillExecutors(Seq("2"))
    +    requestExecutorRunnable.getAllValues.asScala.last.run()
    +    verify(podOperations).delete(SECOND_EXECUTOR_POD)
    +    verify(podOperations, never()).delete(FIRST_EXECUTOR_POD)
    +  }
    +
    +  test("Executors should be requested in batches.") {
    +    sparkConf
    +        .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1)
    +        .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2)
    +    val scheduler = newSchedulerBackend()
    +    scheduler.start()
    +    requestExecutorRunnable.getValue.run()
    +    when(podOperations.create(any(classOf[Pod])))
    +      .thenAnswer(AdditionalAnswers.returnsFirstArg())
    +    expectPodCreationWithId(1, FIRST_EXECUTOR_POD)
    +    expectPodCreationWithId(2, SECOND_EXECUTOR_POD)
    +    allocatorRunnable.getValue.run()
    +    verify(podOperations).create(FIRST_EXECUTOR_POD)
    +    verify(podOperations, never()).create(SECOND_EXECUTOR_POD)
    +    val registerFirstExecutorMessage = RegisterExecutor(
    +        "1", mock[RpcEndpointRef], "localhost", 1, Map.empty[String, String])
    +    when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty)
    +    driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext])
    +        .apply(registerFirstExecutorMessage)
    +    allocatorRunnable.getValue.run()
    +    verify(podOperations).create(SECOND_EXECUTOR_POD)
    +  }
    +
    +  test("Deleting executors and then running an allocator pass after finding the loss reason" +
    +      " should only delete the pod once.") {
    +    sparkConf
    +        .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1)
    +        .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1)
    +    val scheduler = newSchedulerBackend()
    +    scheduler.start()
    +    requestExecutorRunnable.getValue.run()
    +    when(podOperations.create(any(classOf[Pod])))
    +        .thenAnswer(AdditionalAnswers.returnsFirstArg())
    +    expectPodCreationWithId(1, FIRST_EXECUTOR_POD)
    +    allocatorRunnable.getValue.run()
    +    val executorEndpointRef = mock[RpcEndpointRef]
    +    when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000))
    +    val registerFirstExecutorMessage = RegisterExecutor(
    +        "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String])
    +    when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty)
    +    driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext])
    +        .apply(registerFirstExecutorMessage)
    +    scheduler.doRequestTotalExecutors(0)
    +    requestExecutorRunnable.getAllValues.asScala.last.run()
    +    scheduler.doKillExecutors(Seq("1"))
    +    requestExecutorRunnable.getAllValues.asScala.last.run()
    +    verify(podOperations, times(1)).delete(FIRST_EXECUTOR_POD)
    +    driverEndpoint.getValue.onDisconnected(executorEndpointRef.address)
    +
    +    val exitedPod = exitPod(FIRST_EXECUTOR_POD, 0)
    +    executorPodsWatcherArgument.getValue.eventReceived(Action.DELETED, exitedPod)
    +    allocatorRunnable.getValue.run()
    +    verify(podOperations, times(1)).delete(FIRST_EXECUTOR_POD)
    +    verify(driverEndpointRef, times(1)).ask[Boolean](
    +        RemoveExecutor("1", ExecutorExited(
    +            0,
    +            exitCausedByApp = false,
    +            s"Container in pod ${exitedPod.getMetadata.getName} exited from" +
    +              s" explicit termination request.")))
    +  }
    +
    +  test("Executors that disconnect from application errors are noted as exits caused by app.") {
    +    sparkConf
    +        .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1)
    +        .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1)
    +    val scheduler = newSchedulerBackend()
    +    scheduler.start()
    +    expectPodCreationWithId(1, FIRST_EXECUTOR_POD)
    +    when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg())
    +    requestExecutorRunnable.getValue.run()
    +    allocatorRunnable.getValue.run()
    +    val executorEndpointRef = mock[RpcEndpointRef]
    +    when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000))
    +    val registerFirstExecutorMessage = RegisterExecutor(
    +        "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String])
    +    when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty)
    +    driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext])
    +        .apply(registerFirstExecutorMessage)
    +    driverEndpoint.getValue.onDisconnected(executorEndpointRef.address)
    +    executorPodsWatcherArgument.getValue.eventReceived(
    +        Action.ERROR, exitPod(FIRST_EXECUTOR_POD, 1))
    +
    +    expectPodCreationWithId(2, SECOND_EXECUTOR_POD)
    +    scheduler.doRequestTotalExecutors(1)
    +    requestExecutorRunnable.getValue.run()
    +    allocatorRunnable.getAllValues.asScala.last.run()
    +    verify(driverEndpointRef).ask[Boolean](
    +        RemoveExecutor("1", ExecutorExited(
    +            1,
    +            exitCausedByApp = true,
    +            s"Pod ${FIRST_EXECUTOR_POD.getMetadata.getName}'s executor container exited with" +
    +              " exit status code 1.")))
    +    verify(podOperations, never()).delete(FIRST_EXECUTOR_POD)
    +  }
    +
    +  test("Executors should only try to get the loss reason a number of times before giving up and" +
    --- End diff --
    
    name is too long, try to make them fit in one line.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #84265 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84265/testReport)** for PR 19468 at commit [`b85cfc4`](https://github.com/apache/spark/commit/b85cfc4038c8de9340b78d10edf88ab76dd90ba3).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150706815
  
    --- 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 {
    --- End diff --
    
    Don't you need the actual concatenation (and thus iteration over `hostToLocalTaskCount`) to happen inside the synchronized block?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150701280
  
    --- Diff: resource-managers/kubernetes/core/pom.xml ---
    @@ -0,0 +1,102 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent_2.11</artifactId>
    +    <version>2.3.0-SNAPSHOT</version>
    +    <relativePath>../../../pom.xml</relativePath>
    +  </parent>
    +
    +  <artifactId>spark-kubernetes_2.11</artifactId>
    +  <packaging>jar</packaging>
    +  <name>Spark Project Kubernetes</name>
    +  <properties>
    +    <sbt.project.name>kubernetes</sbt.project.name>
    +    <kubernetes.client.version>3.0.0</kubernetes.client.version>
    +  </properties>
    +
    +  <dependencies>
    +    <dependency>
    +      <groupId>org.apache.spark</groupId>
    +      <artifactId>spark-core_${scala.binary.version}</artifactId>
    +      <version>${project.version}</version>
    +    </dependency>
    +
    +    <dependency>
    +      <groupId>org.apache.spark</groupId>
    +      <artifactId>spark-core_${scala.binary.version}</artifactId>
    +      <version>${project.version}</version>
    +      <type>test-jar</type>
    +      <scope>test</scope>
    +    </dependency>
    +
    +    <dependency>
    +      <groupId>io.fabric8</groupId>
    +      <artifactId>kubernetes-client</artifactId>
    +      <version>${kubernetes.client.version}</version>
    +      <exclusions>
    +        <exclusion>
    +          <groupId>com.fasterxml.jackson.core</groupId>
    +          <artifactId>jackson-core</artifactId>
    --- End diff --
    
    I think you can use `*` here instead of multiple exclusions.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153246728
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SPARK_VERSION
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +private[spark] object Config extends Logging {
    +
    +  val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$SPARK_VERSION")
    +
    +  val DOCKER_IMAGE_PULL_POLICY =
    +    ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
    +      .doc("Kubernetes image pull policy. Valid values are Always, Never, and IfNotPresent.")
    +      .stringConf
    +      .checkValues(Set("Always", "Never", "IfNotPresent"))
    +      .createWithDefault("IfNotPresent")
    +
    +  val APISERVER_AUTH_DRIVER_CONF_PREFIX =
    +      "spark.kubernetes.authenticate.driver"
    +  val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
    +      "spark.kubernetes.authenticate.driver.mounted"
    +  val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken"
    +  val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile"
    +  val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile"
    +  val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile"
    +  val CA_CERT_FILE_CONF_SUFFIX = "caCertFile"
    +
    +  val KUBERNETES_SERVICE_ACCOUNT_NAME =
    +    ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName")
    +      .doc("Service account that is used when running the driver pod. The driver pod uses" +
    +        " this service account when requesting executor pods from the API server. If specific" +
    +        " credentials are given for the driver pod to use, the driver will favor" +
    +        " using those credentials instead.")
    +      .stringConf
    +      .createOptional
    +
    +  // Note that while we set a default for this when we start up the
    +  // scheduler, the specific default value is dynamically determined
    +  // based on the executor memory.
    +  val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD =
    +    ConfigBuilder("spark.kubernetes.executor.memoryOverhead")
    +      .doc("The amount of off-heap memory (in megabytes) to be allocated per executor. This" +
    +        " is memory that accounts for things like VM overheads, interned strings, other native" +
    +        " overheads, etc. This tends to grow with the executor size. (typically 6-10%).")
    +      .bytesConf(ByteUnit.MiB)
    +      .createOptional
    --- End diff --
    
    It's in #19717. See https://github.com/apache-spark-on-k8s/spark/blob/60234a29846955b8a6e8cb6fbb1dc35da3c3b4f2/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #84088 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84088/testReport)** for PR 19468 at commit [`b75b413`](https://github.com/apache/spark/commit/b75b4136352d4606a41ce2b3fe1c7e31fdf71ffc).


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r151253653
  
    --- 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 --
    
    Minor also, but I think matching on `Option` is discouraged? Should we use `.map...getOrElse` here?


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    I am actually quite swamped right now, hence the delay in getting to this PR.
    I will try to take a pass over the weekend; but if it does get merged if @vanzin is ok with it, fine by me !


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Thanks everyone and congrats!



---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153089366
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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()
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        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)
    +            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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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()
    --- End diff --
    
    `allocatorExecutor.awaitTermination` ?



---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146943151
  
    --- 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 =>
    --- End diff --
    
    +1 on Throwable - is there a more specific exception/error to catch? or just let it through


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82940/
    Test PASSed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146427690
  
    --- 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 --
    
    To clarify, spark-submit in the above conversation refers to the client-side component of K8s mode, and not actually SparkSubmit in spark-core. 


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146957225
  
    --- Diff: resource-managers/kubernetes/core/pom.xml ---
    @@ -0,0 +1,102 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent_2.11</artifactId>
    +    <version>2.3.0-SNAPSHOT</version>
    +    <relativePath>../../../pom.xml</relativePath>
    +  </parent>
    +
    +  <artifactId>spark-kubernetes_2.11</artifactId>
    +  <packaging>jar</packaging>
    +  <name>Spark Project Kubernetes</name>
    +  <properties>
    +    <sbt.project.name>kubernetes</sbt.project.name>
    +    <kubernetes.client.version>2.2.13</kubernetes.client.version>
    --- End diff --
    
    Any reason why client version is from previous major version and much older minor version ?
    I was looking at https://mvnrepository.com/artifact/io.fabric8/kubernetes-client



---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150881619
  
    --- 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,
    --- End diff --
    
    Removed the checks.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150623867
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala ---
    @@ -0,0 +1,49 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SparkConf
    +
    +private[spark] object ConfigurationUtils {
    +  def parsePrefixedKeyValuePairs(
    +      sparkConf: SparkConf,
    +      prefix: String,
    +      configType: String): Map[String, String] = {
    +    val fromPrefix = sparkConf.getAllWithPrefix(prefix)
    +    fromPrefix.groupBy(_._1).foreach {
    +      case (key, values) =>
    +        require(values.size == 1,
    +          s"Cannot have multiple values for a given $configType key, got key $key with" +
    +            s" values $values")
    +    }
    +    fromPrefix.toMap
    +  }
    +
    +  def requireSecondIfFirstIsDefined(
    --- End diff --
    
    Removed.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #82940 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82940/testReport)** for PR 19468 at commit [`c565c9f`](https://github.com/apache/spark/commit/c565c9ffd7e5371ee4425d69ecaf49ce92199fc7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Thanks for the changes, and all the great work !
    The pr LGTM for merge.
    @rxin and @felixcheung have already +1'ed it.
    
    We can leave it open for a day or so in case others want to do a last pass over it before merging it in.
    +CC @vanzin @ jiangxb1987 @markhamstra - any additional comments ?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150624122
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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 io.fabric8.kubernetes.client.Config
    +
    +import org.apache.spark.SparkContext
    +import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
    +import org.apache.spark.util.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)
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153399402
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SPARK_VERSION
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +private[spark] object Config extends Logging {
    +
    +  val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$SPARK_VERSION")
    --- End diff --
    
    I'm somewhat concerned about this though.
    IMO one of key value we delivery with this feature is running on k8s out of the box, and the Docker images we are going released with Spark is a key aspect of that (again, as discussed, we should highlight that this SPIP/feature is going to deliver and release that)
    
    Secondly, it is going to be very easy to make a mistake and set the "wrong" Docker images (there are many many ways this could go wrong, including mismatch versions of submission client vs driver &/ executor) that would be very hard to diagnose (by no means it will fail fast - worse case you get some subtle correctness issues in data output). And IMO having this version number set consistently by default is really going to help that.
    
    What's the concern with having the default value?
    



---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #84230 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84230/testReport)** for PR 19468 at commit [`a44c29e`](https://github.com/apache/spark/commit/a44c29e9effe3698b5cf948a0fbb12ab16282670).


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153351573
  
    --- 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 =>
    --- End diff --
    
    @liyinan926 this is outstanding, can we remove the catch here?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146961172
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.deploy.k8s
    +
    +import java.io.File
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClient}
    +import io.fabric8.kubernetes.client.utils.HttpClientUtils
    +import okhttp3.Dispatcher
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus common suffixes to
    + * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL
    + * options for different components.
    + */
    +private[spark] object SparkKubernetesClientFactory {
    +
    +  def createKubernetesClient(
    +      master: String,
    +      namespace: Option[String],
    +      kubernetesAuthConfPrefix: String,
    +      sparkConf: SparkConf,
    +      maybeServiceAccountToken: Option[File],
    +      maybeServiceAccountCaCert: Option[File]): KubernetesClient = {
    +    val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX"
    +    val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX"
    +    val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf)
    +      .map(new File(_))
    +      .orElse(maybeServiceAccountToken)
    +    val oauthTokenValue = sparkConf.getOption(oauthTokenConf)
    +    ConfigurationUtils.requireNandDefined(
    +      oauthTokenFile,
    +      oauthTokenValue,
    +      s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" +
    +        s" value $oauthTokenConf.")
    +
    +    val caCertFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX")
    +      .orElse(maybeServiceAccountCaCert.map(_.getAbsolutePath))
    +    val clientKeyFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX")
    --- End diff --
    
    If I understand from the response to @vanzin, this is used in both spark-submit and in driver ?
    I am assuming clientKeyFile is private key for the submiting user ?
    
    Assuming both are true - how is the private key transmitted and secured ? Are there any security concerns ?
    Is it available only at the driver or available/required at executors too ? If yes, any concerns there ?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146989963
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,440 @@
    +/*
    + * 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.JavaConverters._
    +import scala.collection.mutable
    +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 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 = 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())
    +
    +  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
    +    // TODO investigate why Utils.tryLogNonFatalError() doesn't work in this context.
    +    // When using Utils.tryLogNonFatalError some of the code fails but without any logs or
    +    // indication as to why.
    +    try {
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        runningExecutorsToPods.values.foreach(kubernetesClient.pods().delete(_))
    +        runningExecutorsToPods.clear()
    +        runningPodsToExecutors.clear()
    +      }
    +      executorPodsByIPs.clear()
    +      val resource = executorWatchResource.getAndSet(null)
    +      if (resource != null) {
    +        resource.close()
    +      }
    +    } catch {
    +      case e: Throwable => logError("Uncaught exception while shutting down controllers.", e)
    +    }
    +    try {
    +      logInfo("Closing kubernetes client")
    +      kubernetesClient.close()
    +    } catch {
    +      case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e)
    +    }
    +  }
    +
    +  /**
    +   * @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]() ++
    +      KubernetesClusterSchedulerBackend.this.synchronized {
    --- End diff --
    
    I see multiple uses of `KubernetesClusterSchedulerBackend.this` : is this an artifact of earlier code evolution where this was in an inner class/trait ? Simply `synchronized` will do ?


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83061 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83061/testReport)** for PR 19468 at commit [`b0a5839`](https://github.com/apache/spark/commit/b0a5839684d3eb08e1ad93db735440c26b3a1d1e).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #82938 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82938/testReport)** for PR 19468 at commit [`c052212`](https://github.com/apache/spark/commit/c052212888e01eac90a006bfb5d14c513e33d0a3).


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    @felixcheung @mridulm All the comments have been addressed. Is this good to merge?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147013219
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.deploy.k8s
    +
    +import java.io.File
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClient}
    +import io.fabric8.kubernetes.client.utils.HttpClientUtils
    +import okhttp3.Dispatcher
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus common suffixes to
    + * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL
    + * options for different components.
    + */
    +private[spark] object SparkKubernetesClientFactory {
    +
    +  def createKubernetesClient(
    +      master: String,
    +      namespace: Option[String],
    +      kubernetesAuthConfPrefix: String,
    +      sparkConf: SparkConf,
    +      maybeServiceAccountToken: Option[File],
    +      maybeServiceAccountCaCert: Option[File]): KubernetesClient = {
    +    val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX"
    +    val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX"
    +    val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf)
    +      .map(new File(_))
    +      .orElse(maybeServiceAccountToken)
    +    val oauthTokenValue = sparkConf.getOption(oauthTokenConf)
    +    ConfigurationUtils.requireNandDefined(
    +      oauthTokenFile,
    +      oauthTokenValue,
    +      s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" +
    +        s" value $oauthTokenConf.")
    +
    +    val caCertFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX")
    +      .orElse(maybeServiceAccountCaCert.map(_.getAbsolutePath))
    +    val clientKeyFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX")
    --- End diff --
    
    Thanks for clarifying.
    I am assuming the key is used for all requests submitted transparently - that is without any explicit use of it ?


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r148044235
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.deploy.k8s
    +
    +import java.io.File
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClient}
    --- End diff --
    
    The following is recommended:
    ```
    import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient,
      KubernetesClient}
    ```


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83060/
    Test FAILed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146943979
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,440 @@
    +/*
    + * 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.JavaConverters._
    +import scala.collection.mutable
    +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 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 = 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())
    +
    +  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
    +    // TODO investigate why Utils.tryLogNonFatalError() doesn't work in this context.
    +    // When using Utils.tryLogNonFatalError some of the code fails but without any logs or
    +    // indication as to why.
    +    try {
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        runningExecutorsToPods.values.foreach(kubernetesClient.pods().delete(_))
    +        runningExecutorsToPods.clear()
    +        runningPodsToExecutors.clear()
    +      }
    +      executorPodsByIPs.clear()
    +      val resource = executorWatchResource.getAndSet(null)
    +      if (resource != null) {
    +        resource.close()
    +      }
    +    } catch {
    +      case e: Throwable => logError("Uncaught exception while shutting down controllers.", e)
    +    }
    +    try {
    +      logInfo("Closing kubernetes client")
    +      kubernetesClient.close()
    +    } catch {
    +      case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e)
    +    }
    +  }
    +
    +  /**
    +   * @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]() ++
    +      KubernetesClusterSchedulerBackend.this.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]
    +  }
    +
    +  /**
    +   * Allocates a new executor pod
    +   *
    +   * @param nodeToLocalTaskCount  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.
    +   * @return A tuple of the new executor name and the Pod data structure.
    +   */
    +  private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = {
    +    val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString
    +    val executorPod = executorPodFactory.createExecutorPod(
    +      executorId,
    +      applicationId(),
    +      driverUrl,
    +      conf.getExecutorEnv,
    +      driverPod,
    +      nodeToLocalTaskCount)
    +    try {
    +      (executorId, kubernetesClient.pods.create(executorPod))
    +    } catch {
    +      case throwable: Throwable =>
    +        logError("Failed to allocate executor pod.", throwable)
    +        throw throwable
    +    }
    +  }
    +
    +  override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] {
    +    totalExpectedExecutors.set(requestedTotal)
    +    true
    +  }
    +
    +  override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] {
    +    RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +      for (executor <- executorIds) {
    +        val maybeRemovedExecutor = runningExecutorsToPods.remove(executor)
    +        maybeRemovedExecutor.foreach { executorPod =>
    +          kubernetesClient.pods().delete(executorPod)
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod)
    +          runningPodsToExecutors.remove(executorPod.getMetadata.getName)
    +        }
    +        if (maybeRemovedExecutor.isEmpty) {
    +          logWarning(s"Unable to remove pod for unknown executor $executor")
    +        }
    +      }
    +    }
    +    true
    +  }
    +
    +  def getExecutorPodByIP(podIP: String): Option[Pod] = {
    +    // Note: Per https://github.com/databricks/scala-style-guide#concurrency, we don't
    +    // want to be switching to scala.collection.concurrent.Map on
    +    // executorPodsByIPs.
    +    val pod = executorPodsByIPs.get(podIP)
    +    Option(pod)
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      if (action == Action.MODIFIED && pod.getStatus.getPhase == "Running"
    +        && pod.getMetadata.getDeletionTimestamp == null) {
    +        val podIP = pod.getStatus.getPodIP
    +        val clusterNodeName = pod.getSpec.getNodeName
    +        logDebug(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.")
    +        executorPodsByIPs.put(podIP, pod)
    +      } else if ((action == Action.MODIFIED && pod.getMetadata.getDeletionTimestamp != null) ||
    +        action == Action.DELETED || action == Action.ERROR) {
    +        val podName = pod.getMetadata.getName
    +        val podIP = pod.getStatus.getPodIP
    +        logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +        if (podIP != null) {
    +          executorPodsByIPs.remove(podIP)
    +        }
    +        if (action == Action.ERROR) {
    +          logInfo(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason)
    --- End diff --
    
    Warning or Error?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153089664
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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()
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        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)
    +            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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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()
    +      }
    --- End diff --
    
    I am not very sure of the semantics of watcher here - should we close watcher before executor deletes here ?


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82886/
    Test PASSed.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by skonto <gi...@git.apache.org>.
Github user skonto commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    + @susanxhuynh @ArtRand (Mesosphere). 


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    @mridulm It would be great if you could look at this again, thanks!


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153089805
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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()
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        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)
    +            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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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 = synchronized {
    +      mutable.Map[String, Int]() ++ 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 = RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +      executorIds.flatMap { executorId =>
    +        runningExecutorsToPods.remove(executorId) match {
    +          case Some(pod) =>
    +            disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            Some(pod)
    +
    +          case None =>
    +            logWarning(s"Unable to remove pod for unknown executor $executorId")
    +            None
    +        }
    +      }
    +    }
    +
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      val podName = pod.getMetadata.getName
    +      val podIP = pod.getStatus.getPodIP
    +
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +            && pod.getMetadata.getDeletionTimestamp == null) =>
    +          val clusterNodeName = pod.getSpec.getNodeName
    +          logInfo(s"Executor pod $podName ready, launched at $clusterNodeName as IP $podIP.")
    +          executorPodsByIPs.put(podIP, pod)
    +
    +        case Action.DELETED | Action.ERROR =>
    +          val executorId = getExecutorId(pod)
    +          logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +          if (podIP != null) {
    +            executorPodsByIPs.remove(podIP)
    +          }
    +
    +          val executorExitReason = if (action == Action.ERROR) {
    +            logWarning(s"Received error event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnError(pod)
    +          } else if (action == Action.DELETED) {
    +            logWarning(s"Received delete event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnDelete(pod)
    +          } else {
    +            throw new IllegalStateException(
    +              s"Unknown action that should only be DELETED or ERROR: $action")
    +          }
    +          podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason)
    +
    +          if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) {
    +            log.warn(s"Executor with id $executorId was not marked as disconnected, but the" +
    +              s" watch received an event of type $action for this executor. The executor may" +
    +              " have failed to start in the first place and never registered with the driver.")
    +          }
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +
    +        case _ => logDebug(s"Received event of executor pod $podName: " + action)
    +      }
    +    }
    +
    +    override def onClose(cause: KubernetesClientException): Unit = {
    +      logDebug("Executor pod watch closed.", cause)
    +    }
    +
    +    private def getExecutorExitStatus(pod: Pod): Int = {
    +      val containerStatuses = pod.getStatus.getContainerStatuses
    +      if (!containerStatuses.isEmpty) {
    +        // we assume the first container represents the pod status. This assumption may not hold
    +        // true in the future. Revisit this if side-car containers start running inside executor
    +        // pods.
    +        getExecutorExitStatus(containerStatuses.get(0))
    +      } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS
    +    }
    +
    +    private def getExecutorExitStatus(containerStatus: ContainerStatus): Int = {
    +      Option(containerStatus.getState).map { containerState =>
    +        Option(containerState.getTerminated).map { containerStateTerminated =>
    +          containerStateTerminated.getExitCode.intValue()
    +        }.getOrElse(UNKNOWN_EXIT_CODE)
    +      }.getOrElse(UNKNOWN_EXIT_CODE)
    +    }
    +
    +    private def isPodAlreadyReleased(pod: Pod): Boolean = {
    +      val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL)
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        !runningExecutorsToPods.contains(executorId)
    +      }
    +    }
    +
    +    private def executorExitReasonOnError(pod: Pod): ExecutorExited = {
    +      val containerExitStatus = getExecutorExitStatus(pod)
    +      // container was probably actively killed by the driver.
    +      if (isPodAlreadyReleased(pod)) {
    +        ExecutorExited(containerExitStatus, exitCausedByApp = false,
    +          s"Container in pod ${pod.getMetadata.getName} exited from explicit termination" +
    +            " request.")
    +      } else {
    +        val containerExitReason = s"Pod ${pod.getMetadata.getName}'s executor container " +
    +          s"exited with exit status code $containerExitStatus."
    +        ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason)
    +      }
    +    }
    +
    +    private def executorExitReasonOnDelete(pod: Pod): ExecutorExited = {
    +      val exitMessage = if (isPodAlreadyReleased(pod)) {
    +        s"Container in pod ${pod.getMetadata.getName} exited from explicit termination request."
    +      } else {
    +        s"Pod ${pod.getMetadata.getName} deleted or lost."
    +      }
    +      ExecutorExited(getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage)
    +    }
    +
    +    private def getExecutorId(pod: Pod): String = {
    +      val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL)
    +      require(executorId != null, "Unexpected pod metadata; expected all executor pods " +
    +        s"to have label $SPARK_EXECUTOR_ID_LABEL.")
    +      executorId
    +    }
    +  }
    +
    +  override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = {
    +    new KubernetesDriverEndpoint(rpcEnv, properties)
    +  }
    +
    +  private class KubernetesDriverEndpoint(
    +      rpcEnv: RpcEnv,
    +      sparkProperties: Seq[(String, String)])
    +    extends DriverEndpoint(rpcEnv, sparkProperties) {
    +
    +    override def onDisconnected(rpcAddress: RpcAddress): Unit = {
    +      addressToExecutorId.get(rpcAddress).foreach { executorId =>
    +        if (disableExecutor(executorId)) {
    +          RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +            runningExecutorsToPods.get(executorId).foreach { pod =>
    +              disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            }
    +          }
    +        }
    +      }
    +    }
    +  }
    +}
    +
    +private object KubernetesClusterSchedulerBackend {
    +  private val UNKNOWN_EXIT_CODE = -1
    +  // Number of times we are allowed check for the loss reason for an executor before we give up
    +  // and assume the executor failed for good, and attribute it to a framework fault.
    +  val MAX_EXECUTOR_LOST_REASON_CHECKS = 10
    --- End diff --
    
    Is this something that will require customization based on user deployment : for example very overloaded cluster or very aggressive value for `spark.kubernetes.allocation.batch.delay` ?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147001912
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId),
    +      (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    --- End diff --
    
    Hm, the above comment should be more precise.
    
    DNS is handled by kube-dns in any given Kubernetes context. (kube-dns is an optional component, but in future commits it will become clear that Spark will require kube-dns to be installed on a cluster, and we will document as such). kube-dns creates a DNS entry for services that route to the IPs of pods in the system. But kube-dns does not create a DNS entry for every pod, unless there is a service that maps to it.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147025026
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId),
    +      (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    --- End diff --
    
    Thanks for clarifying @mccheah.
    
    One clarification though:
    > IP addresses are strictly managed by the Kubernetes framework, so it's unlikely we're going to run into differences between Ipv4 and Ipv6 in different Kubernetes clusters. We should assume that one of these two address types are being used across all clusters and work with that
    
    When I referred to IPv6 support, I was not referring to interop between IPv4 and IPv6 (that is a different can of worms !). What I wanted to clarify was that IPv6 is not supported as IP (supported via hostnames though).
    If all we are getting are IPv4 ip's, I do not see any obvious issues.
    
    > The Kubernetes code needs to be intelligent about knowing which pods are co-located on the same underlying Kubelet
    
    This would be critical to get HOST locality level working properly. The performance implications of not getting this right would be non trivial.
    
    Looking forward to how this has been addressed !



---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150742085
  
    --- 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))
    --- End diff --
    
    Done.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84083/
    Test FAILed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152373795
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.deploy.k8s
    +
    +import java.io.File
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClient}
    +import io.fabric8.kubernetes.client.utils.HttpClientUtils
    +import okhttp3.Dispatcher
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus common suffixes to
    + * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL
    + * options for different components.
    + */
    +private[spark] object SparkKubernetesClientFactory {
    +
    +  def createKubernetesClient(
    +      master: String,
    +      namespace: Option[String],
    +      kubernetesAuthConfPrefix: String,
    +      sparkConf: SparkConf,
    +      defaultServiceAccountToken: Option[File],
    +      defaultServiceAccountCaCert: Option[File]): KubernetesClient = {
    +    val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX"
    +    val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX"
    +    val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf)
    +      .map(new File(_))
    +      .orElse(defaultServiceAccountToken)
    +    val oauthTokenValue = sparkConf.getOption(oauthTokenConf)
    +    ConfigurationUtils.requireNandDefined(
    +      oauthTokenFile,
    +      oauthTokenValue,
    +      s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a " +
    +        s"value $oauthTokenConf.")
    +
    +    val caCertFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX")
    +      .orElse(defaultServiceAccountCaCert.map(_.getAbsolutePath))
    +    val clientKeyFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX")
    +    val clientCertFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX")
    +    val dispatcher = new Dispatcher(
    +      ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher"))
    +    val config = new ConfigBuilder()
    +      .withApiVersion("v1")
    +      .withMasterUrl(master)
    +      .withWebsocketPingInterval(0)
    +      .withOption(oauthTokenValue) {
    +        (token, configBuilder) => configBuilder.withOauthToken(token)
    +      }.withOption(oauthTokenFile) {
    +        (file, configBuilder) =>
    +            configBuilder.withOauthToken(Files.toString(file, Charsets.UTF_8))
    +      }.withOption(caCertFile) {
    +        (file, configBuilder) => configBuilder.withCaCertFile(file)
    +      }.withOption(clientKeyFile) {
    +        (file, configBuilder) => configBuilder.withClientKeyFile(file)
    +      }.withOption(clientCertFile) {
    +        (file, configBuilder) => configBuilder.withClientCertFile(file)
    +      }.withOption(namespace) {
    +        (ns, configBuilder) => configBuilder.withNamespace(ns)
    +      }.build()
    +    val baseHttpClient = HttpClientUtils.createHttpClient(config)
    +    val httpClientWithCustomDispatcher = baseHttpClient.newBuilder()
    +      .dispatcher(dispatcher)
    +      .build()
    +    new DefaultKubernetesClient(httpClientWithCustomDispatcher, config)
    +  }
    +
    +  private implicit class OptionConfigurableConfigBuilder(configBuilder: ConfigBuilder) {
    --- End diff --
    
    Done.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83057/
    Test FAILed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r148916191
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.deploy.k8s
    +
    +import java.io.File
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClient}
    --- End diff --
    
    @jiangxb1987 Is this the convention across the codebase? I don't think most IDEs will preserve that formatting.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #84116 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84116/testReport)** for PR 19468 at commit [`cb12fec`](https://github.com/apache/spark/commit/cb12fecb9cc8b6686b08ef1e82de3e62f32b4b73).


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152442892
  
    --- Diff: pom.xml ---
    @@ -2648,6 +2648,13 @@
           </modules>
         </profile>
     
    +    <profile>
    +      <id>kubernetes</id>
    +      <modules>
    +        <module>resource-managers/kubernetes/core</module>
    --- End diff --
    
    both yarn and meson don't have a sub-directory called `core`, are we going to add more modules?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153297620
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SPARK_VERSION
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +private[spark] object Config extends Logging {
    +
    +  val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$SPARK_VERSION")
    --- End diff --
    
    I think we should remove this instead of defaulting. WDYT @mccheah 


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152204683
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    --- End diff --
    
    nit: We can remove `s` here.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152219175
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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)
    +          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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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 = synchronized {
    +      mutable.Map[String, Int]() ++ 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 = RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +      executorIds.flatMap { executorId =>
    +        runningExecutorsToPods.remove(executorId) match {
    +          case Some(pod) =>
    +            disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            Some(pod)
    +
    +          case None =>
    +            logWarning(s"Unable to remove pod for unknown executor $executorId")
    +            None
    +        }
    +      }
    +    }
    +
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      val podName = pod.getMetadata.getName
    +      val podIP = pod.getStatus.getPodIP
    +
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +            && pod.getMetadata.getDeletionTimestamp == null) =>
    +          val clusterNodeName = pod.getSpec.getNodeName
    +          logInfo(s"Executor pod $podName ready, launched at $clusterNodeName as IP $podIP.")
    +          executorPodsByIPs.put(podIP, pod)
    +
    +        case Action.DELETED | Action.ERROR =>
    +          val executorId = getExecutorId(pod)
    +          logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +          if (podIP != null) {
    +            executorPodsByIPs.remove(podIP)
    +          }
    +
    +          val executorExitReason = if (action == Action.ERROR) {
    +            logWarning(s"Received error event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnError(pod)
    +          } else if (action == Action.DELETED) {
    +            logWarning(s"Received delete event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnDelete(pod)
    +          } else {
    +            throw new IllegalStateException(
    +              s"Unknown action that should only be DELETED or ERROR: $action")
    +          }
    +          podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason)
    +
    +          if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) {
    +            log.warn(s"Executor with id $executorId was not marked as disconnected, but the" +
    +              s" watch received an event of type $action for this executor. The executor may" +
    +              s" have failed to start in the first place and never registered with the driver.")
    +          }
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +
    +        case _ => logDebug(s"Received event of executor pod $podName: " + action)
    +      }
    +    }
    +
    +    override def onClose(cause: KubernetesClientException): Unit = {
    +      logDebug("Executor pod watch closed.", cause)
    +    }
    +
    +    private def getExecutorExitStatus(pod: Pod): Int = {
    +      val containerStatuses = pod.getStatus.getContainerStatuses
    +      if (!containerStatuses.isEmpty) {
    +        // we assume the first container represents the pod status. This assumption may not hold
    +        // true in the future. Revisit this if side-car containers start running inside executor
    +        // pods.
    +        getExecutorExitStatus(containerStatuses.get(0))
    +      } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS
    +    }
    +
    +    private def getExecutorExitStatus(containerStatus: ContainerStatus): Int = {
    +      Option(containerStatus.getState).map { containerState =>
    +        Option(containerState.getTerminated).map {containerStateTerminated =>
    +          containerStateTerminated.getExitCode.intValue()
    +        }.getOrElse(UNKNOWN_EXIT_CODE)
    +      }.getOrElse(UNKNOWN_EXIT_CODE)
    +    }
    +
    +    private def isPodAlreadyReleased(pod: Pod): Boolean = {
    +      val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL)
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        !runningExecutorsToPods.contains(executorId)
    +      }
    +    }
    +
    +    private def executorExitReasonOnError(pod: Pod): ExecutorExited = {
    +      val containerExitStatus = getExecutorExitStatus(pod)
    +      // container was probably actively killed by the driver.
    +      if (isPodAlreadyReleased(pod)) {
    +        ExecutorExited(containerExitStatus, exitCausedByApp = false,
    +          s"Container in pod ${pod.getMetadata.getName} exited from explicit termination" +
    +            " request.")
    +      } else {
    +        val containerExitReason = s"Pod ${pod.getMetadata.getName}'s executor container " +
    +          s"exited with exit status code $containerExitStatus."
    +        ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason)
    +      }
    +    }
    +
    +    private def executorExitReasonOnDelete(pod: Pod): ExecutorExited = {
    +      val exitMessage = if (isPodAlreadyReleased(pod)) {
    +        s"Container in pod ${pod.getMetadata.getName} exited from explicit termination request."
    +      } else {
    +        s"Pod ${pod.getMetadata.getName} deleted or lost."
    +      }
    +      ExecutorExited(getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage)
    +    }
    +
    +    private def getExecutorId(pod: Pod): String = {
    +      val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL)
    +      require(executorId != null, "Unexpected pod metadata; expected all executor pods " +
    +        s"to have label $SPARK_EXECUTOR_ID_LABEL.")
    +      executorId
    +    }
    +  }
    +
    +  override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = {
    +    new KubernetesDriverEndpoint(rpcEnv, properties)
    +  }
    +
    +  private class KubernetesDriverEndpoint(
    +    rpcEnv: RpcEnv,
    +    sparkProperties: Seq[(String, String)])
    +    extends DriverEndpoint(rpcEnv, sparkProperties) {
    +
    +    override def onDisconnected(rpcAddress: RpcAddress): Unit = {
    +      addressToExecutorId.get(rpcAddress).foreach { executorId =>
    +        if (disableExecutor(executorId)) {
    +          RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +            runningExecutorsToPods.get(executorId).foreach { pod =>
    +              disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            }
    +          }
    +        }
    +      }
    +    }
    +  }
    +}
    +
    +private object KubernetesClusterSchedulerBackend {
    +  private val UNKNOWN_EXIT_CODE = -1
    +  // Number of times we are allowed check for the loss reason for an executor before we give up
    +  // and assume the executor failed for good, and attribute it to a framework fault.
    +  val MAX_EXECUTOR_LOST_REASON_CHECKS = 10
    +}
    +
    --- End diff --
    
    nit: remove an extra line.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152217750
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala ---
    @@ -0,0 +1,439 @@
    +/*
    + * 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.{ExecutorService, ScheduledExecutorService, TimeUnit}
    +
    +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder, PodList}
    +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher}
    +import io.fabric8.kubernetes.client.Watcher.Action
    +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NonNamespaceOperation, PodResource}
    +import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations}
    +import org.mockito.Matchers.{any, eq => mockitoEq}
    +import org.mockito.Mockito.{doNothing, never, times, verify, when}
    +import org.scalatest.BeforeAndAfter
    +import org.scalatest.mock.MockitoSugar._
    +import scala.collection.JavaConverters._
    +import scala.concurrent.Future
    +
    +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.rpc._
    +import org.apache.spark.scheduler.{ExecutorExited, LiveListenerBus, SlaveLost, TaskSchedulerImpl}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +import org.apache.spark.util.ThreadUtils
    +
    +class KubernetesClusterSchedulerBackendSuite
    +    extends SparkFunSuite with BeforeAndAfter {
    +
    +  private val APP_ID = "test-spark-app"
    +  private val DRIVER_POD_NAME = "spark-driver-pod"
    +  private val NAMESPACE = "test-namespace"
    +  private val SPARK_DRIVER_HOST = "localhost"
    +  private val SPARK_DRIVER_PORT = 7077
    +  private val POD_ALLOCATION_INTERVAL = 60L
    +  private val DRIVER_URL = RpcEndpointAddress(
    +      SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
    +  private val FIRST_EXECUTOR_POD = new PodBuilder()
    +    .withNewMetadata()
    +      .withName("pod1")
    +      .endMetadata()
    +    .withNewSpec()
    +      .withNodeName("node1")
    +      .endSpec()
    +    .withNewStatus()
    +      .withHostIP("192.168.99.100")
    +      .endStatus()
    +    .build()
    +  private val SECOND_EXECUTOR_POD = new PodBuilder()
    +    .withNewMetadata()
    +      .withName("pod2")
    +      .endMetadata()
    +    .withNewSpec()
    +      .withNodeName("node2")
    +      .endSpec()
    +    .withNewStatus()
    +      .withHostIP("192.168.99.101")
    +      .endStatus()
    +    .build()
    +
    +  private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]
    +  private type LABELED_PODS = FilterWatchListDeletable[
    +      Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]]
    --- End diff --
    
    ditto.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145253453
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder}
    +import org.apache.commons.io.FilenameUtils
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +// Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    --- End diff --
    
    `/** */` instead of `//`.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #84227 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84227/testReport)** for PR 19468 at commit [`ae396cf`](https://github.com/apache/spark/commit/ae396cf436aa9a2bbc9f73dcb0447a525457240c).


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    @vanzin @jiangxb1987 @mridulm We have addressed all the comments so far. Is this good to merge?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150624079
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala ---
    @@ -0,0 +1,113 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.{SPARK_VERSION => sparkVersion}
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +package object config extends Logging {
    +
    +  private[spark] val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  private[spark] val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$sparkVersion")
    +
    +  private[spark] val DOCKER_IMAGE_PULL_POLICY =
    +    ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
    +      .doc("Docker image pull policy when pulling any docker image in Kubernetes integration")
    +      .stringConf
    +      .createWithDefault("IfNotPresent")
    +
    --- End diff --
    
    Removed.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #84083 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84083/testReport)** for PR 19468 at commit [`7afce3f`](https://github.com/apache/spark/commit/7afce3f1d61df2ecba3efd2019ca4d287fc0bffb).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145251179
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala ---
    @@ -0,0 +1,37 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.internal.Logging
    +
    +private[spark] object ConfigurationUtils extends Logging {
    --- End diff --
    
    `Logging` is not used?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146426033
  
    --- Diff: pom.xml ---
    @@ -2649,6 +2649,13 @@
         </profile>
     
         <profile>
    +      <id>kubernetes</id>
    +      <modules>
    +        <module>resource-managers/kubernetes/core</module>
    --- End diff --
    
    Based on a discussion in last week's meeting with Shane Knapp from RISELab, we want to keep the integration tests as a sub-module here - in the interest of keeping test code together. We should have the additional parent pom to facilitate that.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    I see the latest changes are backported back to the fork - thanks!  it's useful to do since I think we get more comprehensive test coverage there, for now.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145260359
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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.{ExecutorService, ScheduledExecutorService, TimeUnit}
    +
    +import scala.collection.JavaConverters._
    +import scala.concurrent.Future
    +
    +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder, PodList}
    +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher}
    +import io.fabric8.kubernetes.client.Watcher.Action
    +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NonNamespaceOperation, PodResource}
    +import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations}
    +import org.mockito.Matchers.{any, eq => mockitoEq}
    +import org.mockito.Mockito.{mock => _, _}
    +import org.scalatest.BeforeAndAfter
    +import org.scalatest.mock.MockitoSugar._
    +
    +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.rpc._
    +import org.apache.spark.scheduler.{ExecutorExited, LiveListenerBus, SlaveLost, TaskSchedulerImpl}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +private[spark] class KubernetesClusterSchedulerBackendSuite
    +    extends SparkFunSuite with BeforeAndAfter {
    +
    +  private val APP_ID = "test-spark-app"
    +  private val DRIVER_POD_NAME = "spark-driver-pod"
    +  private val NAMESPACE = "test-namespace"
    +  private val SPARK_DRIVER_HOST = "localhost"
    +  private val SPARK_DRIVER_PORT = 7077
    +  private val POD_ALLOCATION_INTERVAL = 60L
    +  private val DRIVER_URL = RpcEndpointAddress(
    +      SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
    +  private val FIRST_EXECUTOR_POD = new PodBuilder()
    +    .withNewMetadata()
    +      .withName("pod1")
    +      .endMetadata()
    +    .withNewSpec()
    +      .withNodeName("node1")
    +      .endSpec()
    +    .withNewStatus()
    +      .withHostIP("192.168.99.100")
    +      .endStatus()
    +    .build()
    +  private val SECOND_EXECUTOR_POD = new PodBuilder()
    +    .withNewMetadata()
    +      .withName("pod2")
    +      .endMetadata()
    +    .withNewSpec()
    +      .withNodeName("node2")
    +      .endSpec()
    +    .withNewStatus()
    +      .withHostIP("192.168.99.101")
    +      .endStatus()
    +    .build()
    +
    +  private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]
    +  private type LABELLED_PODS = FilterWatchListDeletable[
    +      Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]]
    +  private type IN_NAMESPACE_PODS = NonNamespaceOperation[
    +      Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]
    +
    +  @Mock
    +  private var sparkContext: SparkContext = _
    +
    +  @Mock
    +  private var listenerBus: LiveListenerBus = _
    +
    +  @Mock
    +  private var taskSchedulerImpl: TaskSchedulerImpl = _
    +
    +  @Mock
    +  private var allocatorExecutor: ScheduledExecutorService = _
    +
    +  @Mock
    +  private var requestExecutorsService: ExecutorService = _
    +
    +  @Mock
    +  private var executorPodFactory: ExecutorPodFactory = _
    +
    +  @Mock
    +  private var kubernetesClient: KubernetesClient = _
    +
    +  @Mock
    +  private var podOperations: PODS = _
    +
    +  @Mock
    +  private var podsWithLabelOperations: LABELLED_PODS = _
    +
    +  @Mock
    +  private var podsInNamespace: IN_NAMESPACE_PODS = _
    +
    +  @Mock
    +  private var podsWithDriverName: PodResource[Pod, DoneablePod] = _
    +
    +  @Mock
    +  private var rpcEnv: RpcEnv = _
    +
    +  @Mock
    +  private var driverEndpointRef: RpcEndpointRef = _
    +
    +  @Mock
    +  private var executorPodsWatch: Watch = _
    +
    +  private var sparkConf: SparkConf = _
    +  private var executorPodsWatcherArgument: ArgumentCaptor[Watcher[Pod]] = _
    +  private var allocatorRunnable: ArgumentCaptor[Runnable] = _
    +  private var requestExecutorRunnable: ArgumentCaptor[Runnable] = _
    +  private var driverEndpoint: ArgumentCaptor[RpcEndpoint] = _
    +
    +  private val driverPod = new PodBuilder()
    +    .withNewMetadata()
    +      .withName(DRIVER_POD_NAME)
    +      .addToLabels(SPARK_APP_ID_LABEL, APP_ID)
    +      .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_DRIVER_ROLE)
    +      .endMetadata()
    +    .build()
    +
    +  before {
    +    MockitoAnnotations.initMocks(this)
    +    sparkConf = new SparkConf()
    +        .set("spark.app.id", APP_ID)
    +        .set(KUBERNETES_DRIVER_POD_NAME, DRIVER_POD_NAME)
    +        .set(KUBERNETES_NAMESPACE, NAMESPACE)
    +        .set("spark.driver.host", SPARK_DRIVER_HOST)
    +        .set("spark.driver.port", SPARK_DRIVER_PORT.toString)
    +        .set(KUBERNETES_ALLOCATION_BATCH_DELAY, POD_ALLOCATION_INTERVAL)
    +    executorPodsWatcherArgument = ArgumentCaptor.forClass(classOf[Watcher[Pod]])
    +    allocatorRunnable = ArgumentCaptor.forClass(classOf[Runnable])
    +    requestExecutorRunnable = ArgumentCaptor.forClass(classOf[Runnable])
    +    driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint])
    +    when(sparkContext.conf).thenReturn(sparkConf)
    +    when(sparkContext.listenerBus).thenReturn(listenerBus)
    +    when(taskSchedulerImpl.sc).thenReturn(sparkContext)
    +    when(kubernetesClient.pods()).thenReturn(podOperations)
    +    when(podOperations.withLabel(SPARK_APP_ID_LABEL, APP_ID)).thenReturn(podsWithLabelOperations)
    +    when(podsWithLabelOperations.watch(executorPodsWatcherArgument.capture()))
    +        .thenReturn(executorPodsWatch)
    +    when(podOperations.inNamespace(NAMESPACE)).thenReturn(podsInNamespace)
    +    when(podsInNamespace.withName(DRIVER_POD_NAME)).thenReturn(podsWithDriverName)
    +    when(podsWithDriverName.get()).thenReturn(driverPod)
    +    when(allocatorExecutor.scheduleWithFixedDelay(
    +        allocatorRunnable.capture(),
    +        mockitoEq(0L),
    +        mockitoEq(POD_ALLOCATION_INTERVAL),
    +        mockitoEq(TimeUnit.SECONDS))).thenReturn(null)
    +    // Creating Futures in Scala backed by a Java executor service resolves to running
    +    // ExecutorService#execute (as opposed to submit)
    +    doNothing().when(requestExecutorsService).execute(requestExecutorRunnable.capture())
    +    when(rpcEnv.setupEndpoint(
    +        mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture()))
    +        .thenReturn(driverEndpointRef)
    +    when(driverEndpointRef.ask[Boolean]
    +      (any(classOf[Any]))
    +      (any())).thenReturn(mock[Future[Boolean]])
    +  }
    +
    +  test("Basic lifecycle expectations when starting and stopping the scheduler.") {
    +    val scheduler = newSchedulerBackend()
    +    scheduler.start()
    +    assert(executorPodsWatcherArgument.getValue != null)
    +    assert(allocatorRunnable.getValue != null)
    +    scheduler.stop()
    +    verify(executorPodsWatch).close()
    +  }
    +
    +  test("Static allocation should request executors upon first allocator run.") {
    +    sparkConf
    +        .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2)
    +        .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2)
    +    val scheduler = newSchedulerBackend()
    +    scheduler.start()
    +    requestExecutorRunnable.getValue.run()
    +    expectPodCreationWithId(1, FIRST_EXECUTOR_POD)
    +    expectPodCreationWithId(2, SECOND_EXECUTOR_POD)
    +    when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg())
    +    allocatorRunnable.getValue.run()
    +    verify(podOperations).create(FIRST_EXECUTOR_POD)
    +    verify(podOperations).create(SECOND_EXECUTOR_POD)
    +  }
    +
    +  test("Killing executors deletes the executor pods") {
    +    sparkConf
    +        .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2)
    +        .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2)
    +    val scheduler = newSchedulerBackend()
    +    scheduler.start()
    +    requestExecutorRunnable.getValue.run()
    +    expectPodCreationWithId(1, FIRST_EXECUTOR_POD)
    +    expectPodCreationWithId(2, SECOND_EXECUTOR_POD)
    +    when(podOperations.create(any(classOf[Pod])))
    +        .thenAnswer(AdditionalAnswers.returnsFirstArg())
    +    allocatorRunnable.getValue.run()
    +    scheduler.doKillExecutors(Seq("2"))
    +    requestExecutorRunnable.getAllValues.asScala.last.run()
    +    verify(podOperations).delete(SECOND_EXECUTOR_POD)
    +    verify(podOperations, never()).delete(FIRST_EXECUTOR_POD)
    +  }
    +
    +  test("Executors should be requested in batches.") {
    +    sparkConf
    +        .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1)
    +        .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2)
    +    val scheduler = newSchedulerBackend()
    +    scheduler.start()
    +    requestExecutorRunnable.getValue.run()
    +    when(podOperations.create(any(classOf[Pod])))
    +      .thenAnswer(AdditionalAnswers.returnsFirstArg())
    +    expectPodCreationWithId(1, FIRST_EXECUTOR_POD)
    +    expectPodCreationWithId(2, SECOND_EXECUTOR_POD)
    +    allocatorRunnable.getValue.run()
    +    verify(podOperations).create(FIRST_EXECUTOR_POD)
    +    verify(podOperations, never()).create(SECOND_EXECUTOR_POD)
    +    val registerFirstExecutorMessage = RegisterExecutor(
    +        "1", mock[RpcEndpointRef], "localhost", 1, Map.empty[String, String])
    +    when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty)
    +    driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext])
    +        .apply(registerFirstExecutorMessage)
    +    allocatorRunnable.getValue.run()
    +    verify(podOperations).create(SECOND_EXECUTOR_POD)
    +  }
    +
    +  test("Deleting executors and then running an allocator pass after finding the loss reason" +
    --- End diff --
    
    shorter test name?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152450525
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    --- End diff --
    
    Maybe update the doc of this configuration to describe the difference while using Kubernetes backend?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153090634
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,226 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX)
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX)
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX)
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1)
    +  private val executorLimitCores = sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    --- End diff --
    
    How is this getting used ? I see it getting set, but not used anywhere.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146969639
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    --- End diff --
    
    nit: remove applicationId from comment ?
    
    Also, hostnames can go upto 255 chars - labels go upto 63. Unfortunately, I was not sure what `withHostname` expects and what constraints are there on executorPodNamePrefix (can it be an arbitrary string with dns invalid chars ?)


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153083018
  
    --- Diff: resource-managers/kubernetes/core/pom.xml ---
    @@ -0,0 +1,94 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent_2.11</artifactId>
    +    <version>2.3.0-SNAPSHOT</version>
    +    <relativePath>../../../pom.xml</relativePath>
    +  </parent>
    +
    +  <artifactId>spark-kubernetes_2.11</artifactId>
    +  <packaging>jar</packaging>
    +  <name>Spark Project Kubernetes</name>
    +  <properties>
    +    <sbt.project.name>kubernetes</sbt.project.name>
    +    <kubernetes.client.version>3.0.0</kubernetes.client.version>
    +  </properties>
    +
    +  <dependencies>
    +    <dependency>
    +      <groupId>org.apache.spark</groupId>
    +      <artifactId>spark-core_${scala.binary.version}</artifactId>
    +      <version>${project.version}</version>
    +    </dependency>
    +
    +    <dependency>
    +      <groupId>org.apache.spark</groupId>
    +      <artifactId>spark-core_${scala.binary.version}</artifactId>
    +      <version>${project.version}</version>
    +      <type>test-jar</type>
    +      <scope>test</scope>
    +    </dependency>
    +
    +    <dependency>
    +      <groupId>io.fabric8</groupId>
    +      <artifactId>kubernetes-client</artifactId>
    --- End diff --
    
    these dependencies need to be listed, please see
    http://www.apache.org/dev/licensing-howto.html



---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153083057
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.deploy.k8s
    +
    +import java.io.File
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient, KubernetesClient}
    +import io.fabric8.kubernetes.client.utils.HttpClientUtils
    +import okhttp3.Dispatcher
    --- End diff --
    
    this seems new, should this be listed in pom file?



---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152373817
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146976161
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,440 @@
    +/*
    + * 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.JavaConverters._
    +import scala.collection.mutable
    +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]
    --- End diff --
    
    nit: you could use GuardedBy instead of comment.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r151269007
  
    --- 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,
    --- End diff --
    
    I agree that this depends on how clearly integration tests fail on this. But I still don't think this is a good place for such kind of checks. Also as @vanzin pointed out, failing the checks here only stops the scheduled task without stopping the driver/app so it really does not fail fast nor explicitly in real cases.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    @vanzin, you were right, the YARN constants were left overs and made no sense wrt k8s. We discussed it in our weekly meeting - it was simply dead code. I've addressed most of the style comments and the major concern about the constants. It's ready for a more in-depth review.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    @mridulm @vanzin @mateiz @rxin This is waiting for another LGTM I think (aside from the one by @jiangxb1987). If all parties have no major changes to suggest, may I propose we merge this before thanksgiving and revisit it during the review of the [other PR](https://github.com/apache/spark/pull/19717) for K8s support in 2.3? 
    



---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #84096 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84096/testReport)** for PR 19468 at commit [`3b587b4`](https://github.com/apache/spark/commit/3b587b4b4362f184b148c22522821ef7b163717e).


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150704482
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,227 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId)) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    +    ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq
    +    val requiredPorts = Seq(
    +      (EXECUTOR_PORT_NAME, executorPort),
    +      (BLOCK_MANAGER_PORT_NAME, blockManagerPort))
    +      .map(port => {
    --- End diff --
    
    `.map { case (name, port) => `?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146428556
  
    --- Diff: pom.xml ---
    @@ -2649,6 +2649,13 @@
         </profile>
     
         <profile>
    +      <id>kubernetes</id>
    +      <modules>
    +        <module>resource-managers/kubernetes/core</module>
    --- End diff --
    
    It's not absolutely necessary to have integration tests in a specific separate module. However, there are some nice organizational benefits we can get. For example, integration tests in the same module as the core code will need a specific package namespace that is omitted from the `test` phase and only executed in the `integrationTest` phase. Having a separate module means that the integration test pom can just make the `test` phase a no-op and integrationTest runs all tests in the `test` folder. (I don't know if Maven has a concept of a difference between `src/test/scala` and `src/integrationTest/scala`, which would help a lot.)
    
    It's also IMO easier to read the `pom.xml` of the integration test separately from the `pom.xml` of the Kubernetes core implementation. FWIW this is what we have in the integration test POM at the moment: https://github.com/apache-spark-on-k8s/spark/blob/branch-2.2-kubernetes/resource-managers/kubernetes/integration-tests/pom.xml. (The minikube related things are going away with https://github.com/apache-spark-on-k8s/spark/pull/521).
    
    > And that's assuming that you really don't want to run them during unit tests.
    
    We definitely don't want to run these during unit tests - they are relatively expensive, require building Docker images, and require Minikube to be pre-installed on the given machine. Having them in at least the separate integration test phase makes these differences clear.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r151197505
  
    --- 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) {
    --- End diff --
    
    Actually no, the failed driver/executor containers have already finished running and won't take cpu/memory resources. The pod objects of failed driver/executors, however, are not deleted from the API server so users can use `kubectl logs <pod name>` or `kubectl describe pod <pod name>` to check what's going on with the failed pods. So they will take etcd storage but not cpu/memory resources.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150743139
  
    --- 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 {
    +      for (executor <- executorIds) {
    +        val maybeRemovedExecutor = runningExecutorsToPods.remove(executor)
    +        maybeRemovedExecutor.foreach { executorPod =>
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod)
    +          podsToDelete += executorPod
    +        }
    +        if (maybeRemovedExecutor.isEmpty) {
    +          logWarning(s"Unable to remove pod for unknown executor $executor")
    +        }
    +      }
    +    }
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +          && pod.getMetadata.getDeletionTimestamp == null) =>
    +          val podIP = pod.getStatus.getPodIP
    +          val clusterNodeName = pod.getSpec.getNodeName
    +          logInfo(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.")
    +          executorPodsByIPs.put(podIP, pod)
    +        case Action.DELETED | Action.ERROR =>
    +          val executorId = getExecutorId(pod)
    +          val podName = pod.getMetadata.getName
    +          val podIP = pod.getStatus.getPodIP
    +          logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +          if (podIP != null) {
    +            executorPodsByIPs.remove(podIP)
    +          }
    +
    +          val executorExitReason = if (action == Action.ERROR) {
    +            logWarning(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason)
    +            executorExitReasonOnError(pod)
    +          } else if (action == Action.DELETED) {
    +            logWarning(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason)
    +            executorExitReasonOnDelete(pod)
    +          } else {
    +            throw new IllegalStateException(
    +              s"Unknown action that should only be DELETED or ERROR: $action")
    +          }
    +          podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason)
    +
    +          if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) {
    +            log.warn(s"Executor with id $executorId was not marked as disconnected, but the" +
    +              s" watch received an event of type $action for this executor. The executor may" +
    +              s" have failed to start in the first place and never registered with the driver.")
    +          }
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +      }
    +    }
    +
    +    override def onClose(cause: KubernetesClientException): Unit = {
    +      logDebug("Executor pod watch closed.", cause)
    +    }
    +
    +    def getExecutorExitStatus(pod: Pod): Int = {
    +      val containerStatuses = pod.getStatus.getContainerStatuses
    +      if (!containerStatuses.isEmpty) {
    +        // we assume the first container represents the pod status. This assumption may not hold
    +        // true in the future. Revisit this if side-car containers start running inside executor
    +        // pods.
    +        getExecutorExitStatus(containerStatuses.get(0))
    +      } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS
    +    }
    +
    +    def getExecutorExitStatus(containerStatus: ContainerStatus): Int = {
    +      Option(containerStatus.getState).map(containerState =>
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r151198073
  
    --- 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) {
    --- End diff --
    
    Ah, makes sense. It would be good to make the comment explain that.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r148569836
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,227 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    --- End diff --
    
    Why do we need to fetch it? Seems we can always use `executorMemoryMiB.toString` instead.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r148040015
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackendUtils.scala ---
    @@ -0,0 +1,47 @@
    +/*
    + * 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
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.config.{DYN_ALLOCATION_MAX_EXECUTORS, DYN_ALLOCATION_MIN_EXECUTORS, EXECUTOR_INSTANCES}
    +import org.apache.spark.util.Utils
    +
    +private[spark] object SchedulerBackendUtils {
    +  val DEFAULT_NUMBER_EXECUTORS = 2
    --- End diff --
    
    Should we remove this val from `YarnSparkHadoopUtil`?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152446177
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    --- End diff --
    
    Just for reference:
    
    > Limits and requests for memory are measured in bytes. You can express memory as a plain integer or as a fixed-point integer using one of these suffixes: E, P, T, G, M, k. You can also use the power-of-two equivalents: Ei, Pi, Ti, Gi, Mi, Ki.
    
    https://kubernetes.io/docs/concepts/configuration/manage-compute-resources-container/#meaning-of-memory



---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150701656
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SparkConf
    +
    +private[spark] object ConfigurationUtils {
    +
    +  /**
    +   * Extract and parse Spark configuration properties with a given name prefix and
    +   * return the result as a Map. Keys must not have more than one value.
    +   *
    +   * @param sparkConf Spark configuration
    +   * @param prefix the given property name prefix
    +   * @param configType a descriptive note on the type of entities of interest
    +   * @return a Map storing the configuration property keys and values
    +   */
    +  def parsePrefixedKeyValuePairs(
    +      sparkConf: SparkConf,
    +      prefix: String,
    +      configType: String): Map[String, String] = {
    +    val fromPrefix = sparkConf.getAllWithPrefix(prefix)
    +    fromPrefix.groupBy(_._1).foreach {
    +      case (key, values) =>
    +        require(values.size == 1,
    --- End diff --
    
    Can this ever trigger given that `sparkConf` is basically a map?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150494478
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.deploy.k8s
    +
    +import java.io.File
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClient}
    +import io.fabric8.kubernetes.client.utils.HttpClientUtils
    +import okhttp3.Dispatcher
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus common suffixes to
    + * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL
    + * options for different components.
    + */
    +private[spark] object SparkKubernetesClientFactory {
    +
    +  def createKubernetesClient(
    +      master: String,
    +      namespace: Option[String],
    +      kubernetesAuthConfPrefix: String,
    +      sparkConf: SparkConf,
    +      maybeServiceAccountToken: Option[File],
    +      maybeServiceAccountCaCert: Option[File]): KubernetesClient = {
    +    val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX"
    +    val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX"
    +    val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf)
    +      .map(new File(_))
    +      .orElse(maybeServiceAccountToken)
    +    val oauthTokenValue = sparkConf.getOption(oauthTokenConf)
    +    ConfigurationUtils.requireNandDefined(
    +      oauthTokenFile,
    +      oauthTokenValue,
    +      s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" +
    +        s" value $oauthTokenConf.")
    +
    +    val caCertFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX")
    +      .orElse(maybeServiceAccountCaCert.map(_.getAbsolutePath))
    +    val clientKeyFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX")
    +    val clientCertFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX")
    +    val dispatcher = new Dispatcher(
    +      ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher"))
    +    val config = new ConfigBuilder()
    +      .withApiVersion("v1")
    +      .withMasterUrl(master)
    +      .withWebsocketPingInterval(0)
    +      .withOption(oauthTokenValue) {
    +        (token, configBuilder) => configBuilder.withOauthToken(token)
    +      }.withOption(oauthTokenFile) {
    +        (file, configBuilder) =>
    +            configBuilder.withOauthToken(Files.toString(file, Charsets.UTF_8))
    +      }.withOption(caCertFile) {
    +        (file, configBuilder) => configBuilder.withCaCertFile(file)
    +      }.withOption(clientKeyFile) {
    +        (file, configBuilder) => configBuilder.withClientKeyFile(file)
    +      }.withOption(clientCertFile) {
    +        (file, configBuilder) => configBuilder.withClientCertFile(file)
    +      }.withOption(namespace) {
    +        (ns, configBuilder) => configBuilder.withNamespace(ns)
    +      }.build()
    +    val baseHttpClient = HttpClientUtils.createHttpClient(config)
    +    val httpClientWithCustomDispatcher = baseHttpClient.newBuilder()
    +      .dispatcher(dispatcher)
    +      .build()
    +    new DefaultKubernetesClient(httpClientWithCustomDispatcher, config)
    +  }
    +
    +  private implicit class OptionConfigurableConfigBuilder(configBuilder: ConfigBuilder) {
    --- End diff --
    
    Maybe I'm missing something but why is this needed?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147003383
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId),
    +      (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    +    ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq
    +    val requiredPorts = Seq(
    +      (EXECUTOR_PORT_NAME, executorPort),
    +      (BLOCK_MANAGER_PORT_NAME, blockmanagerPort))
    +      .map(port => {
    +        new ContainerPortBuilder()
    +          .withName(port._1)
    +          .withContainerPort(port._2)
    +          .build()
    +      })
    +
    +    val executorContainer = new ContainerBuilder()
    +      .withName(s"executor")
    +      .withImage(executorDockerImage)
    +      .withImagePullPolicy(dockerImagePullPolicy)
    +      .withNewResources()
    +        .addToRequests("memory", executorMemoryQuantity)
    +        .addToLimits("memory", executorMemoryLimitQuantity)
    +        .addToRequests("cpu", executorCpuQuantity)
    +        .endResources()
    +      .addAllToEnv(executorEnv.asJava)
    +      .withPorts(requiredPorts.asJava)
    +      .build()
    +
    +    val executorPod = new PodBuilder()
    +      .withNewMetadata()
    +        .withName(name)
    +        .withLabels(resolvedExecutorLabels.asJava)
    +        .withAnnotations(executorAnnotations.asJava)
    +        .withOwnerReferences()
    +          .addNewOwnerReference()
    +            .withController(true)
    +            .withApiVersion(driverPod.getApiVersion)
    +            .withKind(driverPod.getKind)
    +            .withName(driverPod.getMetadata.getName)
    +            .withUid(driverPod.getMetadata.getUid)
    +            .endOwnerReference()
    +        .endMetadata()
    +      .withNewSpec()
    +        .withHostname(hostname)
    +        .withRestartPolicy("Never")
    --- End diff --
    
    > Am I correct in assuming that each executor will download all the artifacts (jars, files, etc) even if there are others colocated on the same node ?
    
    This will eventually be correct when we introduce the init-container that fetches dependencies before drivers and executors launch.
    
    The notion of locality is a tricky one in the context of Kubernetes. Since Spark drivers and executors are running in a containerized setting, each process is running in a completely isolated setting. This means that multiple pods that even run on the same kubelet will not have any shared storage. This creates some interesting problems that we will address in the future, including that of needing to localize dependencies.
    
    We expect a common workflow for Spark on Kubernetes for users to install dependencies into their driver/executor container's images. This allows one to bypass the overhead of localizing dependencies from remote locations repeatedly. This is especially true if a workload is run multiple times on the same cluster, because the image binary will be cached on the Kubelets in between multiple executions.
    
    But we're also introducing support for Spark applications to be able to localize dependencies from remote locations, such as an HTTP file server or HDFS. That will come in a future commit.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150545828
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,425 @@
    +/*
    + * 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)
    +  require(podAllocationInterval > 0, "Allocation batch delay " +
    --- End diff --
    
    nit: the check can be done in `k8s/config` by `checkValue()`.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r144182701
  
    --- Diff: pom.xml ---
    @@ -2649,6 +2649,13 @@
         </profile>
     
         <profile>
    +      <id>kubernetes</id>
    --- End diff --
    
    We should also change the sbt file to make it work using sbt.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83789 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83789/testReport)** for PR 19468 at commit [`1f271be`](https://github.com/apache/spark/commit/1f271bee5b43c03fab6d2c55d96a93f417cc6915).


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146979317
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,440 @@
    +/*
    + * 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.JavaConverters._
    +import scala.collection.mutable
    +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 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 = 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())
    +
    +  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
    +    // TODO investigate why Utils.tryLogNonFatalError() doesn't work in this context.
    +    // When using Utils.tryLogNonFatalError some of the code fails but without any logs or
    +    // indication as to why.
    +    try {
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        runningExecutorsToPods.values.foreach(kubernetesClient.pods().delete(_))
    +        runningExecutorsToPods.clear()
    +        runningPodsToExecutors.clear()
    +      }
    +      executorPodsByIPs.clear()
    +      val resource = executorWatchResource.getAndSet(null)
    +      if (resource != null) {
    +        resource.close()
    +      }
    +    } catch {
    +      case e: Throwable => logError("Uncaught exception while shutting down controllers.", e)
    +    }
    +    try {
    +      logInfo("Closing kubernetes client")
    +      kubernetesClient.close()
    +    } catch {
    +      case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e)
    +    }
    +  }
    +
    +  /**
    +   * @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]() ++
    +      KubernetesClusterSchedulerBackend.this.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]
    +  }
    +
    +  /**
    +   * Allocates a new executor pod
    +   *
    +   * @param nodeToLocalTaskCount  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.
    +   * @return A tuple of the new executor name and the Pod data structure.
    +   */
    +  private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = {
    +    val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString
    +    val executorPod = executorPodFactory.createExecutorPod(
    +      executorId,
    +      applicationId(),
    +      driverUrl,
    +      conf.getExecutorEnv,
    +      driverPod,
    +      nodeToLocalTaskCount)
    +    try {
    +      (executorId, kubernetesClient.pods.create(executorPod))
    +    } catch {
    +      case throwable: Throwable =>
    +        logError("Failed to allocate executor pod.", throwable)
    +        throw throwable
    +    }
    +  }
    +
    +  override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] {
    +    totalExpectedExecutors.set(requestedTotal)
    +    true
    +  }
    +
    +  override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] {
    +    RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +      for (executor <- executorIds) {
    +        val maybeRemovedExecutor = runningExecutorsToPods.remove(executor)
    +        maybeRemovedExecutor.foreach { executorPod =>
    +          kubernetesClient.pods().delete(executorPod)
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod)
    +          runningPodsToExecutors.remove(executorPod.getMetadata.getName)
    +        }
    +        if (maybeRemovedExecutor.isEmpty) {
    +          logWarning(s"Unable to remove pod for unknown executor $executor")
    +        }
    +      }
    +    }
    +    true
    +  }
    +
    +  def getExecutorPodByIP(podIP: String): Option[Pod] = {
    +    // Note: Per https://github.com/databricks/scala-style-guide#concurrency, we don't
    +    // want to be switching to scala.collection.concurrent.Map on
    +    // executorPodsByIPs.
    --- End diff --
    
    Please remove this comment, or refer to http://spark.apache.org/contributing.html#code-style-guide if relevant.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by ash211 <gi...@git.apache.org>.
Github user ash211 commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Jenkins, ok to test


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r148916566
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,227 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    --- End diff --
    
    More context here: https://github.com/apache-spark-on-k8s/spark/pull/470


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153325898
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SPARK_VERSION
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +private[spark] object Config extends Logging {
    +
    +  val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$SPARK_VERSION")
    +
    +  val DOCKER_IMAGE_PULL_POLICY =
    +    ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
    +      .doc("Kubernetes image pull policy. Valid values are Always, Never, and IfNotPresent.")
    +      .stringConf
    +      .checkValues(Set("Always", "Never", "IfNotPresent"))
    +      .createWithDefault("IfNotPresent")
    +
    +  val APISERVER_AUTH_DRIVER_CONF_PREFIX =
    +      "spark.kubernetes.authenticate.driver"
    +  val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
    +      "spark.kubernetes.authenticate.driver.mounted"
    +  val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken"
    +  val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile"
    +  val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile"
    +  val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile"
    +  val CA_CERT_FILE_CONF_SUFFIX = "caCertFile"
    +
    +  val KUBERNETES_SERVICE_ACCOUNT_NAME =
    +    ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName")
    +      .doc("Service account that is used when running the driver pod. The driver pod uses" +
    +        " this service account when requesting executor pods from the API server. If specific" +
    +        " credentials are given for the driver pod to use, the driver will favor" +
    +        " using those credentials instead.")
    +      .stringConf
    +      .createOptional
    +
    +  // Note that while we set a default for this when we start up the
    +  // scheduler, the specific default value is dynamically determined
    +  // based on the executor memory.
    +  val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD =
    +    ConfigBuilder("spark.kubernetes.executor.memoryOverhead")
    +      .doc("The amount of off-heap memory (in megabytes) to be allocated per executor. This" +
    +        " is memory that accounts for things like VM overheads, interned strings, other native" +
    +        " overheads, etc. This tends to grow with the executor size. (typically 6-10%).")
    +      .bytesConf(ByteUnit.MiB)
    +      .createOptional
    --- End diff --
    
    Perfect !


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152214967
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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)
    +          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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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 = synchronized {
    +      mutable.Map[String, Int]() ++ 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 = RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +      executorIds.flatMap { executorId =>
    +        runningExecutorsToPods.remove(executorId) match {
    +          case Some(pod) =>
    +            disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            Some(pod)
    +
    +          case None =>
    +            logWarning(s"Unable to remove pod for unknown executor $executorId")
    +            None
    +        }
    +      }
    +    }
    +
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      val podName = pod.getMetadata.getName
    +      val podIP = pod.getStatus.getPodIP
    +
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +            && pod.getMetadata.getDeletionTimestamp == null) =>
    +          val clusterNodeName = pod.getSpec.getNodeName
    +          logInfo(s"Executor pod $podName ready, launched at $clusterNodeName as IP $podIP.")
    +          executorPodsByIPs.put(podIP, pod)
    +
    +        case Action.DELETED | Action.ERROR =>
    +          val executorId = getExecutorId(pod)
    +          logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +          if (podIP != null) {
    +            executorPodsByIPs.remove(podIP)
    +          }
    +
    +          val executorExitReason = if (action == Action.ERROR) {
    +            logWarning(s"Received error event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnError(pod)
    +          } else if (action == Action.DELETED) {
    +            logWarning(s"Received delete event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnDelete(pod)
    +          } else {
    +            throw new IllegalStateException(
    +              s"Unknown action that should only be DELETED or ERROR: $action")
    +          }
    +          podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason)
    +
    +          if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) {
    +            log.warn(s"Executor with id $executorId was not marked as disconnected, but the" +
    +              s" watch received an event of type $action for this executor. The executor may" +
    +              s" have failed to start in the first place and never registered with the driver.")
    +          }
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +
    +        case _ => logDebug(s"Received event of executor pod $podName: " + action)
    +      }
    +    }
    +
    +    override def onClose(cause: KubernetesClientException): Unit = {
    +      logDebug("Executor pod watch closed.", cause)
    +    }
    +
    +    private def getExecutorExitStatus(pod: Pod): Int = {
    +      val containerStatuses = pod.getStatus.getContainerStatuses
    +      if (!containerStatuses.isEmpty) {
    +        // we assume the first container represents the pod status. This assumption may not hold
    +        // true in the future. Revisit this if side-car containers start running inside executor
    +        // pods.
    +        getExecutorExitStatus(containerStatuses.get(0))
    +      } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS
    +    }
    +
    +    private def getExecutorExitStatus(containerStatus: ContainerStatus): Int = {
    +      Option(containerStatus.getState).map { containerState =>
    +        Option(containerState.getTerminated).map {containerStateTerminated =>
    --- End diff --
    
    nit: add a space before `containerStateTerminated`.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152633977
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -0,0 +1,114 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SPARK_VERSION
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +private[spark] object Config extends Logging {
    +
    +  val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$SPARK_VERSION")
    +
    +  val DOCKER_IMAGE_PULL_POLICY =
    +    ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
    +      .doc("Docker image pull policy when pulling any docker image in Kubernetes integration")
    +      .stringConf
    +      .createWithDefault("IfNotPresent")
    +
    +  val APISERVER_AUTH_DRIVER_CONF_PREFIX =
    +      "spark.kubernetes.authenticate.driver"
    +  val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
    +      "spark.kubernetes.authenticate.driver.mounted"
    +  val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken"
    +  val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile"
    +  val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile"
    +  val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile"
    +  val CA_CERT_FILE_CONF_SUFFIX = "caCertFile"
    +
    +  val KUBERNETES_SERVICE_ACCOUNT_NAME =
    +    ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName")
    +      .doc("Service account that is used when running the driver pod. The driver pod uses" +
    +        " this service account when requesting executor pods from the API server. If specific" +
    +        " credentials are given for the driver pod to use, the driver will favor" +
    +        " using those credentials instead.")
    +      .stringConf
    +      .createOptional
    +
    +  // Note that while we set a default for this when we start up the
    +  // scheduler, the specific default value is dynamically determined
    +  // based on the executor memory.
    +  val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD =
    +    ConfigBuilder("spark.kubernetes.executor.memoryOverhead")
    +      .doc("The amount of off-heap memory (in megabytes) to be allocated per executor. This" +
    +        " is memory that accounts for things like VM overheads, interned strings, other native" +
    +        " overheads, etc. This tends to grow with the executor size. (typically 6-10%).")
    +      .bytesConf(ByteUnit.MiB)
    +      .createOptional
    +
    +  val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label."
    +  val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation."
    +
    +  val KUBERNETES_DRIVER_POD_NAME =
    +    ConfigBuilder("spark.kubernetes.driver.pod.name")
    +      .doc("Name of the driver pod.")
    +      .stringConf
    +      .createOptional
    +
    +  val KUBERNETES_EXECUTOR_POD_NAME_PREFIX =
    +    ConfigBuilder("spark.kubernetes.executor.podNamePrefix")
    +      .doc("Prefix to use in front of the executor pod names.")
    +      .internal()
    +      .stringConf
    +      .createWithDefault("spark")
    +
    +  val KUBERNETES_ALLOCATION_BATCH_SIZE =
    +    ConfigBuilder("spark.kubernetes.allocation.batch.size")
    +      .doc("Number of pods to launch at once in each round of executor allocation.")
    +      .intConf
    +      .checkValue(value => value > 0, "Allocation batch size should be a positive integer")
    +      .createWithDefault(5)
    +
    +  val KUBERNETES_ALLOCATION_BATCH_DELAY =
    +    ConfigBuilder("spark.kubernetes.allocation.batch.delay")
    +      .doc("Number of seconds to wait between each round of executor allocation.")
    +      .longConf
    +      .checkValue(value => value > 0, s"Allocation batch delay should be a positive integer")
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147015064
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId),
    +      (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    --- End diff --
    
    There are various parts of spark which assert that a hostname is used and not IP - that is, spark makes the assumption about executors and drivers being reachable via hostnames.
    Am I right in understanding that this assumption does not hold for kubernetes case ?
    That is, containers need not have a routable hostname assigned to them ?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r148300912
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,418 @@
    +/*
    + * 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 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
    +  // 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 = 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)
    +  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 " +
    --- End diff --
    
    ditto


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145228122
  
    --- Diff: pom.xml ---
    @@ -2649,6 +2649,13 @@
         </profile>
     
         <profile>
    +      <id>kubernetes</id>
    --- End diff --
    
    Updated. PTAL.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    LGTM, thanks for the awesome work!


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r148042590
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala ---
    @@ -0,0 +1,58 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SparkConf
    +
    +private[spark] object ConfigurationUtils {
    +  def parsePrefixedKeyValuePairs(
    +      sparkConf: SparkConf,
    +      prefix: String,
    +      configType: String): Map[String, String] = {
    +    val fromPrefix = sparkConf.getAllWithPrefix(prefix)
    +    fromPrefix.groupBy(_._1).foreach {
    +      case (key, values) =>
    +        require(values.size == 1,
    +          s"Cannot have multiple values for a given $configType key, got key $key with" +
    +            s" values $values")
    +    }
    +    fromPrefix.toMap
    +  }
    +
    +  def requireBothOrNeitherDefined(
    --- End diff --
    
    Why shall we need the function for this PR? Seems it's just used by `ResourceStagingServerSslOptionsProvider` which is out of scope here.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147014477
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.{SPARK_VERSION => sparkVersion}
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +package object config extends Logging {
    +
    +  private[spark] val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  private[spark] val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$sparkVersion")
    +
    +  private[spark] val DOCKER_IMAGE_PULL_POLICY =
    +    ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
    +      .doc("Docker image pull policy when pulling any docker image in Kubernetes integration")
    +      .stringConf
    +      .createWithDefault("IfNotPresent")
    +
    +
    +  private[spark] val APISERVER_AUTH_DRIVER_CONF_PREFIX =
    +      "spark.kubernetes.authenticate.driver"
    +  private[spark] val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
    +      "spark.kubernetes.authenticate.driver.mounted"
    +  private[spark] val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken"
    +  private[spark] val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile"
    +  private[spark] val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile"
    +  private[spark] val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile"
    +  private[spark] val CA_CERT_FILE_CONF_SUFFIX = "caCertFile"
    +
    +  private[spark] val KUBERNETES_SERVICE_ACCOUNT_NAME =
    +    ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName")
    +      .doc("Service account that is used when running the driver pod. The driver pod uses" +
    +        " this service account when requesting executor pods from the API server. If specific" +
    +        " credentials are given for the driver pod to use, the driver will favor" +
    +        " using those credentials instead.")
    +      .stringConf
    +      .createOptional
    +
    +  // Note that while we set a default for this when we start up the
    +  // scheduler, the specific default value is dynamically determined
    +  // based on the executor memory.
    +  private[spark] val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD =
    +    ConfigBuilder("spark.kubernetes.executor.memoryOverhead")
    +      .doc("The amount of off-heap memory (in megabytes) to be allocated per executor. This" +
    +        " is memory that accounts for things like VM overheads, interned strings, other native" +
    +        " overheads, etc. This tends to grow with the executor size. (typically 6-10%).")
    +      .bytesConf(ByteUnit.MiB)
    +      .createOptional
    +
    +  private[spark] val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label."
    +  private[spark] val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation."
    +
    +  private[spark] val KUBERNETES_DRIVER_POD_NAME =
    +    ConfigBuilder("spark.kubernetes.driver.pod.name")
    +      .doc("Name of the driver pod.")
    +      .stringConf
    +      .createOptional
    +
    +  private[spark] val KUBERNETES_EXECUTOR_POD_NAME_PREFIX =
    +    ConfigBuilder("spark.kubernetes.executor.podNamePrefix")
    +      .doc("Prefix to use in front of the executor pod names.")
    +      .internal()
    +      .stringConf
    +      .createWithDefault("spark")
    +
    +  private[spark] val KUBERNETES_ALLOCATION_BATCH_SIZE =
    +    ConfigBuilder("spark.kubernetes.allocation.batch.size")
    +      .doc("Number of pods to launch at once in each round of dynamic allocation. ")
    +      .intConf
    +      .createWithDefault(5)
    +
    +  private[spark] val KUBERNETES_ALLOCATION_BATCH_DELAY =
    +    ConfigBuilder("spark.kubernetes.allocation.batch.delay")
    +      .doc("Number of seconds to wait between each round of executor allocation. ")
    +      .longConf
    +      .createWithDefault(1)
    +
    +  private[spark] val INIT_CONTAINER_JARS_DOWNLOAD_LOCATION =
    +    ConfigBuilder("spark.kubernetes.mountdependencies.jarsDownloadDir")
    +      .doc("Location to download jars to in the driver and executors. When using" +
    +        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    +        " volume on the driver and executor pod.")
    +      .stringConf
    +      .createWithDefault("/var/spark-data/spark-jars")
    +
    +  private[spark] val KUBERNETES_EXECUTOR_LIMIT_CORES =
    +    ConfigBuilder("spark.kubernetes.executor.limit.cores")
    +      .doc("Specify the hard cpu limit for a single executor pod")
    +      .stringConf
    +      .createOptional
    +
    +  private[spark] val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector."
    +
    +  private[spark] def resolveK8sMaster(rawMasterString: String): String = {
    +    if (!rawMasterString.startsWith("k8s://")) {
    +      throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.")
    +    }
    +    val masterWithoutK8sPrefix = rawMasterString.replaceFirst("k8s://", "")
    +    if (masterWithoutK8sPrefix.startsWith("http://")
    +      || masterWithoutK8sPrefix.startsWith("https://")) {
    --- End diff --
    
    Indeed, you are right - that kind of sucks.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153359040
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -111,5 +111,14 @@ private[spark] object Config extends Logging {
           .stringConf
           .createOptional
     
    +  val KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS =
    +    ConfigBuilder("spark.kubernetes.executor.lostCheck.maxAttempts")
    +      .doc("Maximum number of attempts allowed for checking the reason of an executor loss " +
    +        "before it is assumed that the executor failed.")
    +      .intConf
    +      .checkValue(value => value > 0, "Maximum attempts of checks of executor lost reason " +
    +        "must be a positive integer")
    +      .createWithDefault(5)
    --- End diff --
    
    Wasn't this not 10 earlier ?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145258980
  
    --- 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 --
    
    Why is this method necessary?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r148611431
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,425 @@
    +/*
    + * 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)
    +  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()
    +      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),
    --- End diff --
    
    An alternative to enforcing this contract here is to enforce the contract in some other module. E.g. what about having a trait like so:
    
    ```
    trait ExecutorIdLabelContract {
      // Ensures that the returned pod has the executor ID label if it doesn't exist
      def assignRequiredLabels(executorPod: Pod, executorId: String): Pod
      // Gets the executor ID from a pod using the label that is supposed to have been assigned above
      def getExecutorId(executorPod: Pod): String
    }
    ```
    
    which places the contract in one cohesive place. But for now that seems like over-architecting?


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #82854 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82854/testReport)** for PR 19468 at commit [`75e31a9`](https://github.com/apache/spark/commit/75e31a9f6e52dc4784b3efad1a45c42181cabe4a).
     * This patch **fails to generate documentation**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150708463
  
    --- 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 {
    +      for (executor <- executorIds) {
    +        val maybeRemovedExecutor = runningExecutorsToPods.remove(executor)
    +        maybeRemovedExecutor.foreach { executorPod =>
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod)
    +          podsToDelete += executorPod
    +        }
    +        if (maybeRemovedExecutor.isEmpty) {
    +          logWarning(s"Unable to remove pod for unknown executor $executor")
    +        }
    +      }
    +    }
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +          && pod.getMetadata.getDeletionTimestamp == null) =>
    +          val podIP = pod.getStatus.getPodIP
    +          val clusterNodeName = pod.getSpec.getNodeName
    +          logInfo(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.")
    +          executorPodsByIPs.put(podIP, pod)
    +        case Action.DELETED | Action.ERROR =>
    +          val executorId = getExecutorId(pod)
    +          val podName = pod.getMetadata.getName
    +          val podIP = pod.getStatus.getPodIP
    +          logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +          if (podIP != null) {
    +            executorPodsByIPs.remove(podIP)
    +          }
    +
    +          val executorExitReason = if (action == Action.ERROR) {
    +            logWarning(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason)
    +            executorExitReasonOnError(pod)
    +          } else if (action == Action.DELETED) {
    +            logWarning(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason)
    +            executorExitReasonOnDelete(pod)
    +          } else {
    +            throw new IllegalStateException(
    +              s"Unknown action that should only be DELETED or ERROR: $action")
    +          }
    +          podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason)
    +
    +          if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) {
    +            log.warn(s"Executor with id $executorId was not marked as disconnected, but the" +
    +              s" watch received an event of type $action for this executor. The executor may" +
    +              s" have failed to start in the first place and never registered with the driver.")
    +          }
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +      }
    +    }
    +
    +    override def onClose(cause: KubernetesClientException): Unit = {
    +      logDebug("Executor pod watch closed.", cause)
    +    }
    +
    +    def getExecutorExitStatus(pod: Pod): Int = {
    +      val containerStatuses = pod.getStatus.getContainerStatuses
    +      if (!containerStatuses.isEmpty) {
    +        // we assume the first container represents the pod status. This assumption may not hold
    +        // true in the future. Revisit this if side-car containers start running inside executor
    +        // pods.
    +        getExecutorExitStatus(containerStatuses.get(0))
    +      } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS
    +    }
    +
    +    def getExecutorExitStatus(containerStatus: ContainerStatus): Int = {
    +      Option(containerStatus.getState).map(containerState =>
    +        Option(containerState.getTerminated).map(containerStateTerminated =>
    +          containerStateTerminated.getExitCode.intValue()).getOrElse(UNKNOWN_EXIT_CODE)
    +      ).getOrElse(UNKNOWN_EXIT_CODE)
    +    }
    +
    +    def isPodAlreadyReleased(pod: Pod): Boolean = {
    +      val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL)
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        !runningExecutorsToPods.contains(executorId)
    +      }
    +    }
    +
    +    def executorExitReasonOnError(pod: Pod): ExecutorExited = {
    +      val containerExitStatus = getExecutorExitStatus(pod)
    +      // container was probably actively killed by the driver.
    +      if (isPodAlreadyReleased(pod)) {
    +        ExecutorExited(containerExitStatus, exitCausedByApp = false,
    +          s"Container in pod ${pod.getMetadata.getName} exited from explicit termination" +
    +            " request.")
    +      } else {
    +        val containerExitReason = s"Pod ${pod.getMetadata.getName}'s executor container " +
    +          s"exited with exit status code $containerExitStatus."
    +        ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason)
    +      }
    +    }
    +
    +    def executorExitReasonOnDelete(pod: Pod): ExecutorExited = {
    +      val exitMessage = if (isPodAlreadyReleased(pod)) {
    +        s"Container in pod ${pod.getMetadata.getName} exited from explicit termination request."
    +      } else {
    +        s"Pod ${pod.getMetadata.getName} deleted or lost."
    +      }
    +      ExecutorExited(
    +        getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage)
    --- End diff --
    
    Fits in previous line.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #82940 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82940/testReport)** for PR 19468 at commit [`c565c9f`](https://github.com/apache/spark/commit/c565c9ffd7e5371ee4425d69ecaf49ce92199fc7).


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83425/
    Test PASSed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146421445
  
    --- 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 =>
    --- End diff --
    
    It's probably fine to remove the catch block and let the exception propagate up naturally.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147014641
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    --- End diff --
    
    Sounds good, I wanted to make sure I understood it right that we are making the assumption about port being unbound and available for spark.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153354379
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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()
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        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)
    +            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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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()
    +      }
    --- End diff --
    
    Yes, it depends on if we require the watcher to receive and act on the DELETE events in this case. If not, moving `close` to above the deletes is fine.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153341501
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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()
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        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)
    +            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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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 = synchronized {
    +      mutable.Map[String, Int]() ++ 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 = RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +      executorIds.flatMap { executorId =>
    +        runningExecutorsToPods.remove(executorId) match {
    +          case Some(pod) =>
    +            disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            Some(pod)
    +
    +          case None =>
    +            logWarning(s"Unable to remove pod for unknown executor $executorId")
    +            None
    +        }
    +      }
    +    }
    +
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      val podName = pod.getMetadata.getName
    +      val podIP = pod.getStatus.getPodIP
    +
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +            && pod.getMetadata.getDeletionTimestamp == null) =>
    +          val clusterNodeName = pod.getSpec.getNodeName
    +          logInfo(s"Executor pod $podName ready, launched at $clusterNodeName as IP $podIP.")
    +          executorPodsByIPs.put(podIP, pod)
    +
    +        case Action.DELETED | Action.ERROR =>
    +          val executorId = getExecutorId(pod)
    +          logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +          if (podIP != null) {
    +            executorPodsByIPs.remove(podIP)
    +          }
    +
    +          val executorExitReason = if (action == Action.ERROR) {
    +            logWarning(s"Received error event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnError(pod)
    +          } else if (action == Action.DELETED) {
    +            logWarning(s"Received delete event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnDelete(pod)
    +          } else {
    +            throw new IllegalStateException(
    +              s"Unknown action that should only be DELETED or ERROR: $action")
    +          }
    +          podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason)
    +
    +          if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) {
    +            log.warn(s"Executor with id $executorId was not marked as disconnected, but the" +
    +              s" watch received an event of type $action for this executor. The executor may" +
    +              " have failed to start in the first place and never registered with the driver.")
    +          }
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +
    +        case _ => logDebug(s"Received event of executor pod $podName: " + action)
    +      }
    +    }
    +
    +    override def onClose(cause: KubernetesClientException): Unit = {
    +      logDebug("Executor pod watch closed.", cause)
    +    }
    +
    +    private def getExecutorExitStatus(pod: Pod): Int = {
    +      val containerStatuses = pod.getStatus.getContainerStatuses
    +      if (!containerStatuses.isEmpty) {
    +        // we assume the first container represents the pod status. This assumption may not hold
    +        // true in the future. Revisit this if side-car containers start running inside executor
    +        // pods.
    +        getExecutorExitStatus(containerStatuses.get(0))
    +      } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS
    +    }
    +
    +    private def getExecutorExitStatus(containerStatus: ContainerStatus): Int = {
    +      Option(containerStatus.getState).map { containerState =>
    +        Option(containerState.getTerminated).map { containerStateTerminated =>
    +          containerStateTerminated.getExitCode.intValue()
    +        }.getOrElse(UNKNOWN_EXIT_CODE)
    +      }.getOrElse(UNKNOWN_EXIT_CODE)
    +    }
    +
    +    private def isPodAlreadyReleased(pod: Pod): Boolean = {
    +      val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL)
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        !runningExecutorsToPods.contains(executorId)
    +      }
    +    }
    +
    +    private def executorExitReasonOnError(pod: Pod): ExecutorExited = {
    +      val containerExitStatus = getExecutorExitStatus(pod)
    +      // container was probably actively killed by the driver.
    +      if (isPodAlreadyReleased(pod)) {
    +        ExecutorExited(containerExitStatus, exitCausedByApp = false,
    +          s"Container in pod ${pod.getMetadata.getName} exited from explicit termination" +
    +            " request.")
    +      } else {
    +        val containerExitReason = s"Pod ${pod.getMetadata.getName}'s executor container " +
    +          s"exited with exit status code $containerExitStatus."
    +        ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason)
    +      }
    +    }
    +
    +    private def executorExitReasonOnDelete(pod: Pod): ExecutorExited = {
    +      val exitMessage = if (isPodAlreadyReleased(pod)) {
    +        s"Container in pod ${pod.getMetadata.getName} exited from explicit termination request."
    +      } else {
    +        s"Pod ${pod.getMetadata.getName} deleted or lost."
    +      }
    +      ExecutorExited(getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage)
    +    }
    +
    +    private def getExecutorId(pod: Pod): String = {
    +      val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL)
    +      require(executorId != null, "Unexpected pod metadata; expected all executor pods " +
    +        s"to have label $SPARK_EXECUTOR_ID_LABEL.")
    +      executorId
    +    }
    +  }
    +
    +  override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = {
    +    new KubernetesDriverEndpoint(rpcEnv, properties)
    +  }
    +
    +  private class KubernetesDriverEndpoint(
    +      rpcEnv: RpcEnv,
    +      sparkProperties: Seq[(String, String)])
    +    extends DriverEndpoint(rpcEnv, sparkProperties) {
    +
    +    override def onDisconnected(rpcAddress: RpcAddress): Unit = {
    +      addressToExecutorId.get(rpcAddress).foreach { executorId =>
    +        if (disableExecutor(executorId)) {
    +          RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +            runningExecutorsToPods.get(executorId).foreach { pod =>
    +              disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            }
    +          }
    +        }
    +      }
    +    }
    +  }
    +}
    +
    +private object KubernetesClusterSchedulerBackend {
    +  private val UNKNOWN_EXIT_CODE = -1
    +  // Number of times we are allowed check for the loss reason for an executor before we give up
    +  // and assume the executor failed for good, and attribute it to a framework fault.
    +  val MAX_EXECUTOR_LOST_REASON_CHECKS = 10
    --- End diff --
    
    SGTM - Will add to documentation, thanks!


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153359123
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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()
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        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)
    +            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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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()
    +      }
    --- End diff --
    
    Reversed the order in https://github.com/apache/spark/pull/19468/commits/c3861864efbde824beb722d0cdfe861277086c48.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153331071
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,226 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX)
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX)
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX)
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1)
    +  private val executorLimitCores = sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    --- End diff --
    
    Think we might have been relying on it before but we certainly don't now - this should be removed. Probably want to try it on our fork first and run it through our integration tests to verify this is the case.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    I think the other cases, mesos, yarn/docker are quite different as they are one of the possible environment/setup, whereas in this case there is no other alternative - docker image must be there.
    
    Anyway I think we should proceed as-is and revisit if it makes sense.



---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147015669
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId),
    +      (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    --- End diff --
    
    Containers in Kubernetes will not have resolvable hostnames unless we specifically add a DNS entry for it via a service, which is heavyweight and should be done as infrequently as possible. We will need to handle the cases where this breaks Spark as they come up.
    
    I'm curious as to what some example cases are where the hostname matters - we have tackled a few of them on our fork.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150557217
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,425 @@
    +/*
    + * 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)
    +  require(podAllocationInterval > 0, "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, "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()
    +      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 {
    +      for (executor <- executorIds) {
    +        val maybeRemovedExecutor = runningExecutorsToPods.remove(executor)
    +        maybeRemovedExecutor.foreach { executorPod =>
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod)
    +          podsToDelete += executorPod
    +        }
    +        if (maybeRemovedExecutor.isEmpty) {
    +          logWarning(s"Unable to remove pod for unknown executor $executor")
    +        }
    +      }
    +    }
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    --- End diff --
    
    nit: we can also consider to use case match for `action`, 


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146944230
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,440 @@
    +/*
    + * 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.JavaConverters._
    +import scala.collection.mutable
    +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 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 = 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())
    +
    +  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
    +    // TODO investigate why Utils.tryLogNonFatalError() doesn't work in this context.
    +    // When using Utils.tryLogNonFatalError some of the code fails but without any logs or
    +    // indication as to why.
    +    try {
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        runningExecutorsToPods.values.foreach(kubernetesClient.pods().delete(_))
    +        runningExecutorsToPods.clear()
    +        runningPodsToExecutors.clear()
    +      }
    +      executorPodsByIPs.clear()
    +      val resource = executorWatchResource.getAndSet(null)
    +      if (resource != null) {
    +        resource.close()
    +      }
    +    } catch {
    +      case e: Throwable => logError("Uncaught exception while shutting down controllers.", e)
    +    }
    +    try {
    +      logInfo("Closing kubernetes client")
    +      kubernetesClient.close()
    +    } catch {
    +      case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e)
    +    }
    +  }
    +
    +  /**
    +   * @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]() ++
    +      KubernetesClusterSchedulerBackend.this.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]
    +  }
    +
    +  /**
    +   * Allocates a new executor pod
    +   *
    +   * @param nodeToLocalTaskCount  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.
    +   * @return A tuple of the new executor name and the Pod data structure.
    +   */
    +  private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = {
    +    val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString
    +    val executorPod = executorPodFactory.createExecutorPod(
    +      executorId,
    +      applicationId(),
    +      driverUrl,
    +      conf.getExecutorEnv,
    +      driverPod,
    +      nodeToLocalTaskCount)
    +    try {
    +      (executorId, kubernetesClient.pods.create(executorPod))
    +    } catch {
    +      case throwable: Throwable =>
    +        logError("Failed to allocate executor pod.", throwable)
    +        throw throwable
    +    }
    +  }
    +
    +  override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] {
    +    totalExpectedExecutors.set(requestedTotal)
    +    true
    +  }
    +
    +  override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] {
    +    RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +      for (executor <- executorIds) {
    +        val maybeRemovedExecutor = runningExecutorsToPods.remove(executor)
    +        maybeRemovedExecutor.foreach { executorPod =>
    +          kubernetesClient.pods().delete(executorPod)
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod)
    +          runningPodsToExecutors.remove(executorPod.getMetadata.getName)
    +        }
    +        if (maybeRemovedExecutor.isEmpty) {
    +          logWarning(s"Unable to remove pod for unknown executor $executor")
    +        }
    +      }
    +    }
    +    true
    +  }
    +
    +  def getExecutorPodByIP(podIP: String): Option[Pod] = {
    +    // Note: Per https://github.com/databricks/scala-style-guide#concurrency, we don't
    +    // want to be switching to scala.collection.concurrent.Map on
    +    // executorPodsByIPs.
    +    val pod = executorPodsByIPs.get(podIP)
    +    Option(pod)
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      if (action == Action.MODIFIED && pod.getStatus.getPhase == "Running"
    +        && pod.getMetadata.getDeletionTimestamp == null) {
    +        val podIP = pod.getStatus.getPodIP
    +        val clusterNodeName = pod.getSpec.getNodeName
    +        logDebug(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.")
    +        executorPodsByIPs.put(podIP, pod)
    +      } else if ((action == Action.MODIFIED && pod.getMetadata.getDeletionTimestamp != null) ||
    +        action == Action.DELETED || action == Action.ERROR) {
    +        val podName = pod.getMetadata.getName
    +        val podIP = pod.getStatus.getPodIP
    +        logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +        if (podIP != null) {
    +          executorPodsByIPs.remove(podIP)
    +        }
    +        if (action == Action.ERROR) {
    +          logInfo(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason)
    +          handleErroredPod(pod)
    +        } else if (action == Action.DELETED) {
    +          logInfo(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason)
    +          handleDeletedPod(pod)
    +        }
    +      }
    +    }
    +
    +    override def onClose(cause: KubernetesClientException): Unit = {
    +      logDebug("Executor pod watch closed.", cause)
    +    }
    +
    +    def getExecutorExitStatus(pod: Pod): Int = {
    +      val containerStatuses = pod.getStatus.getContainerStatuses
    +      if (!containerStatuses.isEmpty) {
    +        // we assume the first container represents the pod status. This assumption may not hold
    +        // true in the future. Revisit this if side-car containers start running inside executor
    +        // pods.
    +        getExecutorExitStatus(containerStatuses.get(0))
    +      } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS
    +    }
    +
    +    def getExecutorExitStatus(containerStatus: ContainerStatus): Int = {
    +      Option(containerStatus.getState).map(containerState =>
    +        Option(containerState.getTerminated).map(containerStateTerminated =>
    +          containerStateTerminated.getExitCode.intValue()).getOrElse(UNKNOWN_EXIT_CODE)
    +      ).getOrElse(UNKNOWN_EXIT_CODE)
    +    }
    +
    +    def isPodAlreadyReleased(pod: Pod): Boolean = {
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        !runningPodsToExecutors.contains(pod.getMetadata.getName)
    +      }
    +    }
    +
    +    def handleErroredPod(pod: Pod): Unit = {
    +      val containerExitStatus = getExecutorExitStatus(pod)
    +      // container was probably actively killed by the driver.
    +      val exitReason = if (isPodAlreadyReleased(pod)) {
    +        ExecutorExited(containerExitStatus, exitCausedByApp = false,
    +          s"Container in pod " + pod.getMetadata.getName +
    --- End diff --
    
    `s"Container in pod ${pod.getMetadata.getName}"`


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153349561
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SPARK_VERSION
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +private[spark] object Config extends Logging {
    +
    +  val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$SPARK_VERSION")
    --- End diff --
    
    I agree.  Removed in https://github.com/apache/spark/pull/19468/commits/4bed817e5ab1a26e70050254fec453e839af4c8c.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153277116
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152374715
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala ---
    @@ -0,0 +1,49 @@
    +/*
    + * 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.deploy.k8s
    +
    +package object constants {
    --- End diff --
    
    Done.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #82886 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82886/testReport)** for PR 19468 at commit [`82b79a7`](https://github.com/apache/spark/commit/82b79a7361950ffa6ca545f55628cabeb1b52049).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82938/
    Test PASSed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145255331
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder}
    +import org.apache.commons.io.FilenameUtils
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +// Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    +// common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +      org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs (
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +      "executor label")
    +  require(
    +      !executorLabels.contains(SPARK_APP_ID_LABEL),
    +      s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +      !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +      s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +        s" Spark.")
    +
    +  private val executorAnnotations =
    +      ConfigurationUtils.parsePrefixedKeyValuePairs (
    +          sparkConf,
    +          KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +          "executor annotation")
    +  private val nodeSelector =
    +      ConfigurationUtils.parsePrefixedKeyValuePairs(
    +          sparkConf,
    +          KUBERNETES_NODE_SELECTOR_PREFIX,
    +          "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +      .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +      .get(KUBERNETES_DRIVER_POD_NAME)
    +      .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +      org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +      org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +      .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +      .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +        MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    --- End diff --
    
    double?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152204337
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala ---
    @@ -0,0 +1,49 @@
    +/*
    + * 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.deploy.k8s
    +
    +package object constants {
    --- End diff --
    
    nit: How about making this object `private[spark]` and remove it of each value?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146985776
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,440 @@
    +/*
    + * 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.JavaConverters._
    +import scala.collection.mutable
    +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 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 = 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)
    --- End diff --
    
    Can `delete()` block ? If yes, the actual delete could be done outside the sync block ?
    (Same pattern elsewhere in the code as well - except for cases where we cant remove)


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146425100
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.deploy.k8s
    +
    +import java.io.File
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClient}
    +import io.fabric8.kubernetes.client.utils.HttpClientUtils
    +import okhttp3.Dispatcher
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus common suffixes to
    + * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL
    + * options for different components.
    + */
    +private[spark] object SparkKubernetesClientFactory {
    +
    +  def createKubernetesClient(
    +      master: String,
    +      namespace: Option[String],
    +      kubernetesAuthConfPrefix: String,
    +      sparkConf: SparkConf,
    +      maybeServiceAccountToken: Option[File],
    +      maybeServiceAccountCaCert: Option[File]): KubernetesClient = {
    +    val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX"
    +    val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX"
    +    val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf)
    --- End diff --
    
    This lacks context from the `spark-submit` implementation that is not in this PR.
    
    We intend to have two different sets of authentication options for the Kubernetes API. The first is the credentials for creating a driver pod and all the Kubernetes resources that the application requires outside of executor pods. The second is a set of credentials that the driver can use to create executor pods. These options will have shared suffixes in the configuration keys but different prefixes.
    
    The reasoning for two sets of credentials is twofold:
    
    - The driver needs strictly fewer privileges than `spark-submit`, because the driver only creates + deletes pods but `spark-submit` needs to make pods and other Kubernetes resources. Two sets of credentials allows the driver to have an appropriately limited scope of API access.
    - Part of the credentials includes TLS certificates for accessing the Kubernetes API over HTTPs. A common environment is to have the Kubernetes API server be accessible from a proxy into the cluster from an outside location, but then the driver will access the API server from inside the cluster. A front door for the API server typically asks for a different certificate than the certificate one would present when accessing the API server internally.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153084210
  
    --- Diff: resource-managers/kubernetes/core/pom.xml ---
    @@ -0,0 +1,94 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent_2.11</artifactId>
    +    <version>2.3.0-SNAPSHOT</version>
    +    <relativePath>../../../pom.xml</relativePath>
    +  </parent>
    +
    +  <artifactId>spark-kubernetes_2.11</artifactId>
    +  <packaging>jar</packaging>
    +  <name>Spark Project Kubernetes</name>
    +  <properties>
    +    <sbt.project.name>kubernetes</sbt.project.name>
    +    <kubernetes.client.version>3.0.0</kubernetes.client.version>
    --- End diff --
    
    Last time I asked about the client version, there were concerns regarding maturity/stability of 3.x compared to the 2.2 (iirc) version which was in use - where they resolved ?
    I, obviously, prefer moving to the latest major version; but want to understand the risks involved - if any.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153360441
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -111,5 +111,14 @@ private[spark] object Config extends Logging {
           .stringConf
           .createOptional
     
    +  val KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS =
    +    ConfigBuilder("spark.kubernetes.executor.lostCheck.maxAttempts")
    +      .doc("Maximum number of attempts allowed for checking the reason of an executor loss " +
    +        "before it is assumed that the executor failed.")
    +      .intConf
    +      .checkValue(value => value > 0, "Maximum attempts of checks of executor lost reason " +
    +        "must be a positive integer")
    +      .createWithDefault(5)
    --- End diff --
    
    Sure, changed back to 10.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152217789
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala ---
    @@ -0,0 +1,439 @@
    +/*
    + * 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.{ExecutorService, ScheduledExecutorService, TimeUnit}
    +
    +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder, PodList}
    +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher}
    +import io.fabric8.kubernetes.client.Watcher.Action
    +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NonNamespaceOperation, PodResource}
    +import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations}
    +import org.mockito.Matchers.{any, eq => mockitoEq}
    +import org.mockito.Mockito.{doNothing, never, times, verify, when}
    +import org.scalatest.BeforeAndAfter
    +import org.scalatest.mock.MockitoSugar._
    +import scala.collection.JavaConverters._
    +import scala.concurrent.Future
    +
    +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.rpc._
    +import org.apache.spark.scheduler.{ExecutorExited, LiveListenerBus, SlaveLost, TaskSchedulerImpl}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +import org.apache.spark.util.ThreadUtils
    +
    +class KubernetesClusterSchedulerBackendSuite
    +    extends SparkFunSuite with BeforeAndAfter {
    +
    +  private val APP_ID = "test-spark-app"
    +  private val DRIVER_POD_NAME = "spark-driver-pod"
    +  private val NAMESPACE = "test-namespace"
    +  private val SPARK_DRIVER_HOST = "localhost"
    +  private val SPARK_DRIVER_PORT = 7077
    +  private val POD_ALLOCATION_INTERVAL = 60L
    +  private val DRIVER_URL = RpcEndpointAddress(
    +      SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
    +  private val FIRST_EXECUTOR_POD = new PodBuilder()
    +    .withNewMetadata()
    +      .withName("pod1")
    +      .endMetadata()
    +    .withNewSpec()
    +      .withNodeName("node1")
    +      .endSpec()
    +    .withNewStatus()
    +      .withHostIP("192.168.99.100")
    +      .endStatus()
    +    .build()
    +  private val SECOND_EXECUTOR_POD = new PodBuilder()
    +    .withNewMetadata()
    +      .withName("pod2")
    +      .endMetadata()
    +    .withNewSpec()
    +      .withNodeName("node2")
    +      .endSpec()
    +    .withNewStatus()
    +      .withHostIP("192.168.99.101")
    +      .endStatus()
    +    .build()
    +
    +  private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]
    +  private type LABELED_PODS = FilterWatchListDeletable[
    +      Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]]
    +  private type IN_NAMESPACE_PODS = NonNamespaceOperation[
    +      Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]
    --- End diff --
    
    ditto.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150741898
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.deploy.k8s
    +
    +import java.io.File
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClient}
    +import io.fabric8.kubernetes.client.utils.HttpClientUtils
    +import okhttp3.Dispatcher
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus common suffixes to
    + * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL
    + * options for different components.
    + */
    +private[spark] object SparkKubernetesClientFactory {
    +
    +  def createKubernetesClient(
    +      master: String,
    +      namespace: Option[String],
    +      kubernetesAuthConfPrefix: String,
    +      sparkConf: SparkConf,
    +      maybeServiceAccountToken: Option[File],
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146074603
  
    --- 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 --
    
    Noted, thanks! (I meant "job" in the context of a "batch/streaming job", but I understand that's an overloaded term wrt Spark.)


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84084/
    Test FAILed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152458551
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala ---
    @@ -0,0 +1,43 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SparkConf
    +
    +private[spark] object ConfigurationUtils {
    +
    +  /**
    +   * Extract and parse Spark configuration properties with a given name prefix and
    +   * return the result as a Map. Keys must not have more than one value.
    +   *
    +   * @param sparkConf Spark configuration
    +   * @param prefix the given property name prefix
    +   * @param configType a descriptive note on the type of entities of interest
    +   * @return a Map storing the configuration property keys and values
    +   */
    +  def parsePrefixedKeyValuePairs(
    +      sparkConf: SparkConf,
    +      prefix: String,
    +      configType: String): Map[String, String] = {
    --- End diff --
    
    We are not really using it in the context of this PR. Removed this parameter.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Hi! I think we have quorum at this point.
    Not sure if folks are around to review during the holiday, but I plan to do another final round within a day or so, and merge unless anyone has any concern then.



---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152634018
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,429 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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)
    +          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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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) }
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152633966
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.deploy.k8s
    +
    +import java.io.File
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient, KubernetesClient}
    +import io.fabric8.kubernetes.client.utils.HttpClientUtils
    +import okhttp3.Dispatcher
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus common suffixes to
    + * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL
    + * options for different components.
    + */
    +private[spark] object SparkKubernetesClientFactory {
    +
    +  def createKubernetesClient(
    +      master: String,
    +      namespace: Option[String],
    +      kubernetesAuthConfPrefix: String,
    +      sparkConf: SparkConf,
    +      defaultServiceAccountToken: Option[File],
    +      defaultServiceAccountCaCert: Option[File]): KubernetesClient = {
    +    val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX"
    +    val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX"
    +    val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf)
    +      .map(new File(_))
    +      .orElse(defaultServiceAccountToken)
    +    val oauthTokenValue = sparkConf.getOption(oauthTokenConf)
    +    ConfigurationUtils.requireNandDefined(
    +      oauthTokenFile,
    +      oauthTokenValue,
    +      s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a " +
    +        s"value $oauthTokenConf.")
    +
    +    val caCertFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX")
    +      .orElse(defaultServiceAccountCaCert.map(_.getAbsolutePath))
    +    val clientKeyFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX")
    +    val clientCertFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX")
    +    val dispatcher = new Dispatcher(
    +      ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher"))
    +    val config = new ConfigBuilder()
    +      .withApiVersion("v1")
    +      .withMasterUrl(master)
    +      .withWebsocketPingInterval(0)
    +      .withOption(oauthTokenValue) {
    +        (token, configBuilder) => configBuilder.withOauthToken(token)
    +      }.withOption(oauthTokenFile) {
    +        (file, configBuilder) =>
    +            configBuilder.withOauthToken(Files.toString(file, Charsets.UTF_8))
    +      }.withOption(caCertFile) {
    +        (file, configBuilder) => configBuilder.withCaCertFile(file)
    +      }.withOption(clientKeyFile) {
    +        (file, configBuilder) => configBuilder.withClientKeyFile(file)
    +      }.withOption(clientCertFile) {
    +        (file, configBuilder) => configBuilder.withClientCertFile(file)
    +      }.withOption(namespace) {
    +        (ns, configBuilder) => configBuilder.withNamespace(ns)
    +      }.build()
    +    val baseHttpClient = HttpClientUtils.createHttpClient(config)
    +    val httpClientWithCustomDispatcher = baseHttpClient.newBuilder()
    +      .dispatcher(dispatcher)
    +      .build()
    +    new DefaultKubernetesClient(httpClientWithCustomDispatcher, config)
    +  }
    +
    +  private implicit class OptionConfigurableConfigBuilder(val configBuilder: ConfigBuilder)
    +    extends AnyVal {
    +
    +    def withOption[T]
    +    (option: Option[T])
    +    (configurator: ((T, ConfigBuilder) => ConfigBuilder)): ConfigBuilder = {
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152522621
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,429 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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)
    +          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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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) }
    --- End diff --
    
    nit: break line before `}`.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152373908
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala ---
    @@ -0,0 +1,439 @@
    +/*
    + * 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.{ExecutorService, ScheduledExecutorService, TimeUnit}
    +
    +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder, PodList}
    +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher}
    +import io.fabric8.kubernetes.client.Watcher.Action
    +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NonNamespaceOperation, PodResource}
    +import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations}
    +import org.mockito.Matchers.{any, eq => mockitoEq}
    +import org.mockito.Mockito.{doNothing, never, times, verify, when}
    +import org.scalatest.BeforeAndAfter
    +import org.scalatest.mock.MockitoSugar._
    +import scala.collection.JavaConverters._
    +import scala.concurrent.Future
    +
    +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.rpc._
    +import org.apache.spark.scheduler.{ExecutorExited, LiveListenerBus, SlaveLost, TaskSchedulerImpl}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +import org.apache.spark.util.ThreadUtils
    +
    +class KubernetesClusterSchedulerBackendSuite
    +    extends SparkFunSuite with BeforeAndAfter {
    +
    +  private val APP_ID = "test-spark-app"
    +  private val DRIVER_POD_NAME = "spark-driver-pod"
    +  private val NAMESPACE = "test-namespace"
    +  private val SPARK_DRIVER_HOST = "localhost"
    +  private val SPARK_DRIVER_PORT = 7077
    +  private val POD_ALLOCATION_INTERVAL = 60L
    +  private val DRIVER_URL = RpcEndpointAddress(
    +      SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
    +  private val FIRST_EXECUTOR_POD = new PodBuilder()
    +    .withNewMetadata()
    +      .withName("pod1")
    +      .endMetadata()
    +    .withNewSpec()
    +      .withNodeName("node1")
    +      .endSpec()
    +    .withNewStatus()
    +      .withHostIP("192.168.99.100")
    +      .endStatus()
    +    .build()
    +  private val SECOND_EXECUTOR_POD = new PodBuilder()
    +    .withNewMetadata()
    +      .withName("pod2")
    +      .endMetadata()
    +    .withNewSpec()
    +      .withNodeName("node2")
    +      .endSpec()
    +    .withNewStatus()
    +      .withHostIP("192.168.99.101")
    +      .endStatus()
    +    .build()
    +
    +  private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]
    +  private type LABELED_PODS = FilterWatchListDeletable[
    +      Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]]
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153294451
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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()
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        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)
    +            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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from the API server.
    +            if (executorExited.exitCausedByApp) {
    --- End diff --
    
    We delete the in-memory executor object from `runningExecutorsToPods` in both cases. If `executorExited.exitCausedByApp` is true, we just don't delete the executor object from the Kubernetes API server. Like being explained above, failed/terminated executor pods don't take cpu/memory resources, although they are kept around in etcd so users can check what's going on through the `kubectl logs` and `kubectl describe pod` commands. Hope this addresses your concern.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150957209
  
    --- 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,
    --- End diff --
    
    It's the future code change that these requirements are meant to protect against such that the integration tests fail.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146926738
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,440 @@
    +/*
    + * 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.JavaConverters._
    +import scala.collection.mutable
    +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)
    --- End diff --
    
    nit: don't need `s"` this and previous line, and L98, L103.
    typically we avoid that for (minor) performance reasons


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153333958
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,226 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX)
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX)
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX)
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1)
    +  private val executorLimitCores = sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId)) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    +    ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq
    +    val requiredPorts = Seq(
    +      (EXECUTOR_PORT_NAME, executorPort),
    +      (BLOCK_MANAGER_PORT_NAME, blockManagerPort))
    +      .map { case (name, port) =>
    +        new ContainerPortBuilder()
    +          .withName(name)
    +          .withContainerPort(port)
    +          .build()
    +      }
    +
    +    val executorContainer = new ContainerBuilder()
    +      .withName("executor")
    +      .withImage(executorDockerImage)
    +      .withImagePullPolicy(dockerImagePullPolicy)
    +      .withNewResources()
    +        .addToRequests("memory", executorMemoryQuantity)
    +        .addToLimits("memory", executorMemoryLimitQuantity)
    +        .addToRequests("cpu", executorCpuQuantity)
    +        .endResources()
    +      .addAllToEnv(executorEnv.asJava)
    +      .withPorts(requiredPorts.asJava)
    +      .build()
    +
    +    val executorPod = new PodBuilder()
    +      .withNewMetadata()
    +        .withName(name)
    +        .withLabels(resolvedExecutorLabels.asJava)
    +        .withAnnotations(executorAnnotations.asJava)
    +        .withOwnerReferences()
    +          .addNewOwnerReference()
    +            .withController(true)
    +            .withApiVersion(driverPod.getApiVersion)
    +            .withKind(driverPod.getKind)
    +            .withName(driverPod.getMetadata.getName)
    +            .withUid(driverPod.getMetadata.getUid)
    +            .endOwnerReference()
    +        .endMetadata()
    +      .withNewSpec()
    +        .withHostname(hostname)
    +        .withRestartPolicy("Never")
    +        .withNodeSelector(nodeSelector.asJava)
    +        .endSpec()
    +      .build()
    +
    +    val containerWithExecutorLimitCores = executorLimitCores.map { limitCores =>
    +      val executorCpuLimitQuantity = new QuantityBuilder(false)
    +        .withAmount(limitCores)
    +        .build()
    +      new ContainerBuilder(executorContainer)
    +        .editResources()
    +        .addToLimits("cpu", executorCpuLimitQuantity)
    +        .endResources()
    +        .build()
    +    }.getOrElse(executorContainer)
    +
    +    new PodBuilder(executorPod)
    +      .editSpec()
    +        .addToContainers(containerWithExecutorLimitCores)
    +        .endSpec()
    +      .build()
    +  }
    --- End diff --
    
    Thanks for clarifying, that sounds fine - do we have a `docs/running-on-kubernetes.md` ?
    Would be good idea to list these design choices, restrictions and suggestions there to model user expectations.
    For reference, there is a `docs/running-on-yarn.md` and `docs/running-on-mesos.md`.
    
    Ofcourse, this does not need to be part of this PR ! Just something to add in a followup PR before 2.3


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153350771
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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()
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        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)
    +            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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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()
    +      }
    --- End diff --
    
    What about `kubernetesClient.pods().delete()` ? Is it an async call ? Will it invoke watcher before returning ?
    My impression was that watcher invocation would be deferred.
    
    If this is the case, in the current code, we will not have watcher being notified of the nodes delete's.
    Well, some could, some need not be - essentially a race condition.
    Do we require the watcher to be notified ?
    If yes, then we will need to make this more robust.
    If we do not depend on it - then pushing resource.close() to before the delete will prevent a bunch of unnecessary work.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #82854 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82854/testReport)** for PR 19468 at commit [`75e31a9`](https://github.com/apache/spark/commit/75e31a9f6e52dc4784b3efad1a45c42181cabe4a).


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83100 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83100/testReport)** for PR 19468 at commit [`2b5dcac`](https://github.com/apache/spark/commit/2b5dcac4f188b8c61b67aec131a1eb7d91968356).


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146429113
  
    --- Diff: pom.xml ---
    @@ -2649,6 +2649,13 @@
         </profile>
     
         <profile>
    +      <id>kubernetes</id>
    +      <modules>
    +        <module>resource-managers/kubernetes/core</module>
    --- End diff --
    
    That (keeping them separate) is actually pretty useful for SBT.



---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147211956
  
    --- Diff: resource-managers/kubernetes/core/pom.xml ---
    @@ -0,0 +1,102 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent_2.11</artifactId>
    +    <version>2.3.0-SNAPSHOT</version>
    +    <relativePath>../../../pom.xml</relativePath>
    +  </parent>
    +
    +  <artifactId>spark-kubernetes_2.11</artifactId>
    +  <packaging>jar</packaging>
    +  <name>Spark Project Kubernetes</name>
    +  <properties>
    +    <sbt.project.name>kubernetes</sbt.project.name>
    +    <kubernetes.client.version>2.2.13</kubernetes.client.version>
    --- End diff --
    
    No reason, except that newer versions have not been tested as much.
    
    We recently updated the client on our fork as well. https://github.com/apache-spark-on-k8s/spark/pull/528/files. We're waiting to see that there aren't any regressions and then will update this PR as well.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    For future pull requests, can you create subtasks under https://issues.apache.org/jira/browse/SPARK-18278 ?



---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147018411
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId),
    +      (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    --- End diff --
    
    I forgot what the exact reasons were when I introduced this (way back in 0.6 iirc).
    In particular ipv4 -> ipv6 was one of the culprits iirc (which is why we check for ':' in hostname, but not enforce avoidance of dotted decimal pattern) which was messing up host:port parsing.
    IIRC there were other corner cases which caused issues too - particularly with locality parsing, multi-honed machines, etc.



---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147019793
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId),
    +      (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    --- End diff --
    
    Kubernetes networking is a pretty interesting topic in and of itself, and I foresee part of the challenge of the project will be dealing with these unexpected and unpredictable situations where Spark's assumptions about networking won't hold in the Kubernetes setting. My two cents is that we can only make headway on this project by using what we know to be true about Kubernetes networking - the fact that we can't use hostnames for all executors and have to communicate over IP addresses - and to then tackle networking bugs as they come up. On both production clusters and our integration test environment, we have caught some of these issues as they have come up and patched them accordingly already, and we can continue to do so.
    
    @mridulm - is this a reasonable way forward, or how would you suggest we proceed otherwise?
    
    @foxish do you have any thoughts on this discussion thread?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145254588
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder}
    +import org.apache.commons.io.FilenameUtils
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +// Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    +// common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +      org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs (
    --- End diff --
    
    No space before parenthesis.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153084631
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SPARK_VERSION
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +private[spark] object Config extends Logging {
    +
    +  val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$SPARK_VERSION")
    --- End diff --
    
    Should this default `spark-executor:$SPARK_VERSION` be present ? I would expect the image name to be always explicitly specified by admin (in spark-defaults) or overridden by user.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83060 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83060/testReport)** for PR 19468 at commit [`992acbe`](https://github.com/apache/spark/commit/992acbee3730e966543832ed08318e4ed1424064).


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #84228 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84228/testReport)** for PR 19468 at commit [`f8e3249`](https://github.com/apache/spark/commit/f8e3249d7321bd0d950c252919cb499bed571af9).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153360149
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -111,5 +111,14 @@ private[spark] object Config extends Logging {
           .stringConf
           .createOptional
     
    +  val KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS =
    +    ConfigBuilder("spark.kubernetes.executor.lostCheck.maxAttempts")
    +      .doc("Maximum number of attempts allowed for checking the reason of an executor loss " +
    +        "before it is assumed that the executor failed.")
    +      .intConf
    +      .checkValue(value => value > 0, "Maximum attempts of checks of executor lost reason " +
    +        "must be a positive integer")
    +      .createWithDefault(5)
    --- End diff --
    
    We should probably keep it the same as before, as it's what we've been running with already.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146973714
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId),
    +      (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    --- End diff --
    
    Can we replace use of ip with hostname ?
    Typically within spark, hostnames are used and not ip's.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147024086
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId),
    +      (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    --- End diff --
    
    @mridulm  In response to each of your four points:
    
    - IPv6: IP addresses are strictly managed by the Kubernetes framework, so it's unlikely we're going to run into differences between Ipv4 and Ipv6 in different Kubernetes clusters. We should assume that one of these two address types are being used across all clusters and work with that. So far I've only seen Kubernetes assign IPv4 addresses.
    - No support for multihomed machines of multi-routable IP's: Again, since Kubernetes is managing the IP address and routability of pods, we can understand what the framework will do and work with that. I don't think the framework does anything fancy in this space, but maybe @foxish or others have ideas?
    - Distributed stores: We've thought about this and have some work done on our fork in regards to this that we will eventually contribute back here. @kimoonkim has done some work on this. The short version of the situation we've had to work around is that the container that runs the Spark processes now has a different IP address from the HDFS node that might be colocated with it physically.
    - Rack awareness - @kimoonkim has also done work on this, and similar concerns to the point above have come up.
    
    As a general note, locality is non-trivial in Kubernetes because no two pods will ever share the same IP address, and pods do not share the same IP address as the physical host that is running it. The Kubernetes code needs to be intelligent about knowing which pods are co-located on the same underlying Kubelet. And of course, it's reasonable to believe that the above four considerations are not exhaustive, but we'll address unforeseen factors as they come up.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152218513
  
    --- Diff: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala ---
    @@ -22,15 +22,14 @@ import java.util.concurrent._
     import java.util.concurrent.atomic.AtomicInteger
     import java.util.regex.Pattern
     
    -import scala.collection.JavaConverters._
    -import scala.collection.mutable
    -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue}
    -import scala.util.control.NonFatal
    -
     import org.apache.hadoop.yarn.api.records._
     import org.apache.hadoop.yarn.client.api.AMRMClient
     import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
     import org.apache.hadoop.yarn.conf.YarnConfiguration
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue}
    +import scala.util.control.NonFatal
    --- End diff --
    
    Maybe we should revert here.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145254648
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder}
    +import org.apache.commons.io.FilenameUtils
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +// Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    +// common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +      org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs (
    +      sparkConf,
    --- End diff --
    
    We only double indent parameter declarations. This happens in a bunch of places.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #84234 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84234/testReport)** for PR 19468 at commit [`c386186`](https://github.com/apache/spark/commit/c3861864efbde824beb722d0cdfe861277086c48).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by markhamstra <gi...@git.apache.org>.
Github user markhamstra commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146072523
  
    --- 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 --
    
    ...execution units belonging to a particular Spark Application.
    
    not job


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r146423831
  
    --- 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 --
    
    But since we didn't include the spark-submit implementation in this PR, this implementation of `applicationId()` lacks context.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150705809
  
    --- 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,
    --- End diff --
    
    What's the behavior you want when these checks trigger?
    
    Right now this will just stop this periodic task (which is the the behavior of `ScheduledExecutorService` when a task throws an exception). It won't stop the application or do anything else, so you'll probably end up with a stuck app or some other weird state.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152443965
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala ---
    @@ -0,0 +1,43 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SparkConf
    +
    +private[spark] object ConfigurationUtils {
    +
    +  /**
    +   * Extract and parse Spark configuration properties with a given name prefix and
    +   * return the result as a Map. Keys must not have more than one value.
    +   *
    +   * @param sparkConf Spark configuration
    +   * @param prefix the given property name prefix
    +   * @param configType a descriptive note on the type of entities of interest
    +   * @return a Map storing the configuration property keys and values
    +   */
    +  def parsePrefixedKeyValuePairs(
    +      sparkConf: SparkConf,
    +      prefix: String,
    +      configType: String): Map[String, String] = {
    --- End diff --
    
    where do we use this parameter?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153352222
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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()
    +      RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +        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)
    +            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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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()
    +      }
    --- End diff --
    
    Ah ok - looking up a bit I think we want the order to be reversed here. We should first close the watch to ensure we don't get any deleted events, then delete the pods themselves. We probably want to ensure the pods are deleted even if we fail to close the watch.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153400419
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.deploy.k8s
    +
    +import org.apache.spark.SPARK_VERSION
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config.ConfigBuilder
    +import org.apache.spark.network.util.ByteUnit
    +
    +private[spark] object Config extends Logging {
    +
    +  val KUBERNETES_NAMESPACE =
    +    ConfigBuilder("spark.kubernetes.namespace")
    +      .doc("The namespace that will be used for running the driver and executor pods. When using" +
    +        " spark-submit in cluster mode, this can also be passed to spark-submit via the" +
    +        " --kubernetes-namespace command line argument.")
    +      .stringConf
    +      .createWithDefault("default")
    +
    +  val EXECUTOR_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.executor.docker.image")
    +      .doc("Docker image to use for the executors. Specify this using the standard Docker tag" +
    +        " format.")
    +      .stringConf
    +      .createWithDefault(s"spark-executor:$SPARK_VERSION")
    --- End diff --
    
    The concern is the availability of this default image. If we can make a strong guarantee by making tagging and publication of the images part of the release process, this will not be an issue. @mridulm @foxish @mccheah thoughts?


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150521440
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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 io.fabric8.kubernetes.client.Config
    +
    +import org.apache.spark.SparkContext
    +import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
    +import org.apache.spark.util.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)
    --- End diff --
    
    nit:
    ```
    override def createSchedulerBackend(
        sc: SparkContext,
        masterURL: String,
        scheduler: TaskScheduler): SchedulerBackend = {
    ```


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150743127
  
    --- 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 {
    +      for (executor <- executorIds) {
    +        val maybeRemovedExecutor = runningExecutorsToPods.remove(executor)
    +        maybeRemovedExecutor.foreach { executorPod =>
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod)
    +          podsToDelete += executorPod
    +        }
    +        if (maybeRemovedExecutor.isEmpty) {
    +          logWarning(s"Unable to remove pod for unknown executor $executor")
    +        }
    +      }
    +    }
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +          && pod.getMetadata.getDeletionTimestamp == null) =>
    +          val podIP = pod.getStatus.getPodIP
    +          val clusterNodeName = pod.getSpec.getNodeName
    +          logInfo(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.")
    +          executorPodsByIPs.put(podIP, pod)
    +        case Action.DELETED | Action.ERROR =>
    +          val executorId = getExecutorId(pod)
    +          val podName = pod.getMetadata.getName
    +          val podIP = pod.getStatus.getPodIP
    +          logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +          if (podIP != null) {
    +            executorPodsByIPs.remove(podIP)
    +          }
    +
    +          val executorExitReason = if (action == Action.ERROR) {
    +            logWarning(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason)
    +            executorExitReasonOnError(pod)
    +          } else if (action == Action.DELETED) {
    +            logWarning(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason)
    +            executorExitReasonOnDelete(pod)
    +          } else {
    +            throw new IllegalStateException(
    +              s"Unknown action that should only be DELETED or ERROR: $action")
    +          }
    +          podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason)
    +
    +          if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) {
    +            log.warn(s"Executor with id $executorId was not marked as disconnected, but the" +
    +              s" watch received an event of type $action for this executor. The executor may" +
    +              s" have failed to start in the first place and never registered with the driver.")
    +          }
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +      }
    +    }
    +
    +    override def onClose(cause: KubernetesClientException): Unit = {
    +      logDebug("Executor pod watch closed.", cause)
    +    }
    +
    +    def getExecutorExitStatus(pod: Pod): Int = {
    --- End diff --
    
    Done.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    **[Test build #83903 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83903/testReport)** for PR 19468 at commit [`7f14b71`](https://github.com/apache/spark/commit/7f14b71c0254553893ac70642d0f0b77bb7687a2).


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152631296
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    --- End diff --
    
    Actually we use that to set environment variable `SPARK_EXECUTOR_MEMORY` in the executor container, which is then used to set `spark.executor.memory`.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by foxish <gi...@git.apache.org>.
Github user foxish commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150752598
  
    --- 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) {
    --- End diff --
    
    There are 2 different cases here. If the application succeeds, then as @liyinan926 said, we'll clean up all executors in `stop`. However, if for some reason, the application fails, then we will keep the failed executors and the failed driver around for debugging - this is the intended behavior here.
    
    There are [owner references](https://kubernetes.io/docs/api-reference/v1.8/#ownerreference-v1-meta) between the driver pod and executors, such that the driver pod is the root of the ownership graph of all resources associated with a spark application. So, deleting the driver from the API automatically removes all executors. So, post-inspection, the expectation is that the user will delete the driver to simply clean up the application entirely. (We will include this in documentation)
    
    In the future (not in the Spark 2.3 timeframe), the root of the owner reference graph may be a [CRD representing a Spark Application](https://github.com/apache-spark-on-k8s/spark/issues/111) which will provide more transparency to help manage resources associated with spark applications more intuitively.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147015304
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    + * common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    + */
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +    "executor label")
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      s" Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +      "executor annotation")
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX,
    +      "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +    .get(KUBERNETES_DRIVER_POD_NAME)
    +    .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId and applicationId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    +    val resolvedExecutorLabels = Map(
    +      SPARK_EXECUTOR_ID_LABEL -> executorId,
    +      SPARK_APP_ID_LABEL -> applicationId,
    +      SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
    +      executorLabels
    +    val executorMemoryQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryMiB}Mi")
    +      .build()
    +    val executorMemoryLimitQuantity = new QuantityBuilder(false)
    +      .withAmount(s"${executorMemoryWithOverhead}Mi")
    +      .build()
    +    val executorCpuQuantity = new QuantityBuilder(false)
    +      .withAmount(executorCores.toString)
    +      .build()
    +    val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
    +        .withValue(cp)
    +        .build()
    +    }
    +    val executorExtraJavaOptionsEnv = sparkConf
    +      .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
    +      .map { opts =>
    +        val delimitedOpts = Utils.splitCommandString(opts)
    +        delimitedOpts.zipWithIndex.map {
    +          case (opt, index) =>
    +            new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
    +        }
    +      }.getOrElse(Seq.empty[EnvVar])
    +    val executorEnv = (Seq(
    +      (ENV_EXECUTOR_PORT, executorPort.toString),
    +      (ENV_DRIVER_URL, driverUrl),
    +      // Executor backend expects integral value for executor cores, so round it up to an int.
    +      (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
    +      (ENV_EXECUTOR_MEMORY, executorMemoryString),
    +      (ENV_APPLICATION_ID, applicationId),
    +      (ENV_EXECUTOR_ID, executorId),
    +      (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs)
    +      .map(env => new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +      ) ++ Seq(
    +      new EnvVarBuilder()
    +        .withName(ENV_EXECUTOR_POD_IP)
    +        .withValueFrom(new EnvVarSourceBuilder()
    +          .withNewFieldRef("v1", "status.podIP")
    +          .build())
    +        .build()
    --- End diff --
    
    Just to clarify, the hostname need not be stable - but needs to be stable only for the duration the container is 'alive' : and is not immediately reused (in case some other executor container comes up with exact same hostname after previous exits).


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84230/
    Test PASSed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150708127
  
    --- 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 {
    +      for (executor <- executorIds) {
    +        val maybeRemovedExecutor = runningExecutorsToPods.remove(executor)
    +        maybeRemovedExecutor.foreach { executorPod =>
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod)
    +          podsToDelete += executorPod
    +        }
    +        if (maybeRemovedExecutor.isEmpty) {
    +          logWarning(s"Unable to remove pod for unknown executor $executor")
    +        }
    +      }
    +    }
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +          && pod.getMetadata.getDeletionTimestamp == null) =>
    --- End diff --
    
    Indent one extra level.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153350064
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,226 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX)
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX)
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX)
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1)
    +  private val executorLimitCores = sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    --- End diff --
    
    Agree with @mccheah that we should try it out in our fork. If it turns out we don't really need to set it, we can remove it in a follow-up PR so this won't block this PR from being merged. @mridulm What do you think? 


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150624057
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.deploy.k8s
    +
    +import java.io.File
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClient}
    +import io.fabric8.kubernetes.client.utils.HttpClientUtils
    +import okhttp3.Dispatcher
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus common suffixes to
    + * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL
    + * options for different components.
    + */
    +private[spark] object SparkKubernetesClientFactory {
    +
    +  def createKubernetesClient(
    +      master: String,
    +      namespace: Option[String],
    +      kubernetesAuthConfPrefix: String,
    +      sparkConf: SparkConf,
    +      maybeServiceAccountToken: Option[File],
    +      maybeServiceAccountCaCert: Option[File]): KubernetesClient = {
    +    val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX"
    +    val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX"
    +    val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf)
    +      .map(new File(_))
    +      .orElse(maybeServiceAccountToken)
    +    val oauthTokenValue = sparkConf.getOption(oauthTokenConf)
    +    ConfigurationUtils.requireNandDefined(
    +      oauthTokenFile,
    +      oauthTokenValue,
    +      s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" +
    +        s" value $oauthTokenConf.")
    +
    +    val caCertFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX")
    +      .orElse(maybeServiceAccountCaCert.map(_.getAbsolutePath))
    +    val clientKeyFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX")
    +    val clientCertFile = sparkConf
    +      .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX")
    +    val dispatcher = new Dispatcher(
    +      ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher"))
    +    val config = new ConfigBuilder()
    +      .withApiVersion("v1")
    +      .withMasterUrl(master)
    +      .withWebsocketPingInterval(0)
    +      .withOption(oauthTokenValue) {
    +        (token, configBuilder) => configBuilder.withOauthToken(token)
    +      }.withOption(oauthTokenFile) {
    +        (file, configBuilder) =>
    +            configBuilder.withOauthToken(Files.toString(file, Charsets.UTF_8))
    +      }.withOption(caCertFile) {
    +        (file, configBuilder) => configBuilder.withCaCertFile(file)
    +      }.withOption(clientKeyFile) {
    +        (file, configBuilder) => configBuilder.withClientKeyFile(file)
    +      }.withOption(clientCertFile) {
    +        (file, configBuilder) => configBuilder.withClientCertFile(file)
    +      }.withOption(namespace) {
    +        (ns, configBuilder) => configBuilder.withNamespace(ns)
    +      }.build()
    +    val baseHttpClient = HttpClientUtils.createHttpClient(config)
    +    val httpClientWithCustomDispatcher = baseHttpClient.newBuilder()
    +      .dispatcher(dispatcher)
    +      .build()
    +    new DefaultKubernetesClient(httpClientWithCustomDispatcher, config)
    +  }
    +
    +  private implicit class OptionConfigurableConfigBuilder(configBuilder: ConfigBuilder) {
    --- End diff --
    
    This is needed for the use of `withOption` above.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153087493
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,226 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX)
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX)
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX)
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1)
    +  private val executorLimitCores = sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of name contains
    +    // executorId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    --- End diff --
    
    What is hostname used for here ?
    Based on previous discussion, there is no support for hostname and kubernetes support is only IP based - given this, why not simply set it to some arbitrary random string and not depend on `name` ? (The comment about 63 chars is still incorrect btw - hostnames go upto 255, labels go upto 63 : I think I mentioned this before).
    Or is this to keep the option open in future for dns support ?


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84114/
    Test PASSed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r145255435
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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 scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder}
    +import org.apache.commons.io.FilenameUtils
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.config._
    +import org.apache.spark.deploy.k8s.constants._
    +import org.apache.spark.util.Utils
    +
    +// Configures executor pods. Construct one of these with a SparkConf to set up properties that are
    +// common across all executors. Then, pass in dynamic parameters into createExecutorPod.
    +private[spark] trait ExecutorPodFactory {
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath = sparkConf.get(
    +      org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +  private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
    +
    +  private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs (
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_LABEL_PREFIX,
    +      "executor label")
    +  require(
    +      !executorLabels.contains(SPARK_APP_ID_LABEL),
    +      s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.")
    +  require(
    +      !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +      s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
    +        s" Spark.")
    +
    +  private val executorAnnotations =
    +      ConfigurationUtils.parsePrefixedKeyValuePairs (
    +          sparkConf,
    +          KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
    +          "executor annotation")
    +  private val nodeSelector =
    +      ConfigurationUtils.parsePrefixedKeyValuePairs(
    +          sparkConf,
    +          KUBERNETES_NODE_SELECTOR_PREFIX,
    +          "node selector")
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
    +  private val blockmanagerPort = sparkConf
    +      .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +  private val kubernetesDriverPodName = sparkConf
    +      .get(KUBERNETES_DRIVER_POD_NAME)
    +      .getOrElse(throw new SparkException("Must specify the driver pod name"))
    +
    +  private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +      org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +      org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +      .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +      .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +        MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d)
    +  private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
    --- End diff --
    
    `sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES)`


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150745701
  
    --- 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 {
    --- End diff --
    
    Moved the concatenation inside `synchronized`.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82856/
    Test PASSed.


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r150705036
  
    --- 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)
    --- End diff --
    
    Why protected?


---

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


[GitHub] spark issue #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - Basic Sc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19468
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84088/
    Test PASSed.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by liyinan926 <gi...@git.apache.org>.
Github user liyinan926 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r152373862
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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()
    +
    +  protected override val minRegisteredRatio =
    +    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
    +      0.8
    +    } else {
    +      super.minRegisteredRatio
    +    }
    +
    +  private val executorWatchResource = new AtomicReference[Closeable]
    +  private 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)
    +          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 don't delete the pod running the executor that has an exit condition caused by
    +            // the application from the Kubernetes API server. This allows users to debug later on
    +            // through commands such as "kubectl logs <pod name>" and
    +            // "kubectl describe pod <pod name>". Note that exited containers have terminated and
    +            // therefore won't take CPU and memory resources.
    +            // Otherwise, the executor pod is marked to be deleted from 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 = synchronized {
    +      mutable.Map[String, Int]() ++ 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 = RUNNING_EXECUTOR_PODS_LOCK.synchronized {
    +      executorIds.flatMap { executorId =>
    +        runningExecutorsToPods.remove(executorId) match {
    +          case Some(pod) =>
    +            disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +            Some(pod)
    +
    +          case None =>
    +            logWarning(s"Unable to remove pod for unknown executor $executorId")
    +            None
    +        }
    +      }
    +    }
    +
    +    kubernetesClient.pods().delete(podsToDelete: _*)
    +    true
    +  }
    +
    +  private class ExecutorPodsWatcher extends Watcher[Pod] {
    +
    +    private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
    +
    +    override def eventReceived(action: Action, pod: Pod): Unit = {
    +      val podName = pod.getMetadata.getName
    +      val podIP = pod.getStatus.getPodIP
    +
    +      action match {
    +        case Action.MODIFIED if (pod.getStatus.getPhase == "Running"
    +            && pod.getMetadata.getDeletionTimestamp == null) =>
    +          val clusterNodeName = pod.getSpec.getNodeName
    +          logInfo(s"Executor pod $podName ready, launched at $clusterNodeName as IP $podIP.")
    +          executorPodsByIPs.put(podIP, pod)
    +
    +        case Action.DELETED | Action.ERROR =>
    +          val executorId = getExecutorId(pod)
    +          logDebug(s"Executor pod $podName at IP $podIP was at $action.")
    +          if (podIP != null) {
    +            executorPodsByIPs.remove(podIP)
    +          }
    +
    +          val executorExitReason = if (action == Action.ERROR) {
    +            logWarning(s"Received error event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnError(pod)
    +          } else if (action == Action.DELETED) {
    +            logWarning(s"Received delete event of executor pod $podName. Reason: " +
    +              pod.getStatus.getReason)
    +            executorExitReasonOnDelete(pod)
    +          } else {
    +            throw new IllegalStateException(
    +              s"Unknown action that should only be DELETED or ERROR: $action")
    +          }
    +          podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason)
    +
    +          if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) {
    +            log.warn(s"Executor with id $executorId was not marked as disconnected, but the" +
    +              s" watch received an event of type $action for this executor. The executor may" +
    +              s" have failed to start in the first place and never registered with the driver.")
    +          }
    +          disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod)
    +
    +        case _ => logDebug(s"Received event of executor pod $podName: " + action)
    +      }
    +    }
    +
    +    override def onClose(cause: KubernetesClientException): Unit = {
    +      logDebug("Executor pod watch closed.", cause)
    +    }
    +
    +    private def getExecutorExitStatus(pod: Pod): Int = {
    +      val containerStatuses = pod.getStatus.getContainerStatuses
    +      if (!containerStatuses.isEmpty) {
    +        // we assume the first container represents the pod status. This assumption may not hold
    +        // true in the future. Revisit this if side-car containers start running inside executor
    +        // pods.
    +        getExecutorExitStatus(containerStatuses.get(0))
    +      } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS
    +    }
    +
    +    private def getExecutorExitStatus(containerStatus: ContainerStatus): Int = {
    +      Option(containerStatus.getState).map { containerState =>
    +        Option(containerState.getTerminated).map {containerStateTerminated =>
    --- End diff --
    
    Done.


---

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


[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r147019065
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala ---
    @@ -0,0 +1,440 @@
    +/*
    + * 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.JavaConverters._
    +import scala.collection.mutable
    +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 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 = 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")
    --- End diff --
    
    The errors would be benign given state of current code, but it is better to avoid logical errors.
    I should have given example of `totalExpectedExecutors` instead of `totalRegisteredExecutors` to make things clearer. It is used twice - and there can be change in its value between both uses.



---

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