From 310263c253a8c4a3748cab5b5a7698e076695cd6 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 18 May 2018 13:39:47 -0700 Subject: [PATCH 01/47] [SPARK-24248][K8S] Use the Kubernetes API to populate an event queue for scheduling Previously, the scheduler backend was maintaining state in many places, not only for reading state but also writing to it. For example, state had to be managed in both the watch and in the executor allocator runnable. Furthermore one had to keep track of multiple hash tables. We can do better here by: (1) Consolidating the places where we manage state. Here, we take inspiration from traditional Kubernetes controllers. These controllers tend to implement an event queue which is populated by two sources: a watch connection, and a periodic poller. Controllers typically use both mechanisms for redundancy; the watch connection may drop, so the periodic polling serves as a backup. Both sources write pod updates to a single event queue and then a processor periodically processes the current state of pods as reported by the two sources. (2) Storing less specialized in-memory state in general. Previously we were creating hash tables to represent the state of executors. Instead, it's easier to represent state solely by the event queue, which has predictable read/write patterns and is more or less just a local up-to-date cache of the cluster's status. --- pom.xml | 6 + resource-managers/kubernetes/core/pom.xml | 6 + .../k8s/ExecutorPodsEventHandler.scala | 212 +++++++++ .../k8s/ExecutorPodsPollingEventSource.scala | 58 +++ .../k8s/ExecutorPodsWatchEventSource.scala | 63 +++ .../k8s/KubernetesClusterManager.scala | 27 +- .../KubernetesClusterSchedulerBackend.scala | 406 ++-------------- ...bernetesClusterSchedulerBackendSuite.scala | 449 ------------------ 8 files changed, 410 insertions(+), 817 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala diff --git a/pom.xml b/pom.xml index 6e37e518d86e4..43054371e0ad5 100644 --- a/pom.xml +++ b/pom.xml @@ -150,6 +150,7 @@ 4.5.4 4.4.8 + 3.0.1 3.1 3.4.1 @@ -434,6 +435,11 @@ commons-collections ${commons.collections.version} + + net.sf.trove4j + trove4j + ${trove.version} + org.apache.ivy ivy diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index a62f271273465..79a8d9b0524d8 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -70,6 +70,12 @@ ${fasterxml.jackson.version} + + net.sf.trove4j + trove4j + ${trove.version} + + com.google.guava diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala new file mode 100644 index 0000000000000..40712a9c2a2fa --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala @@ -0,0 +1,212 @@ +/* + * 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.{Future, LinkedBlockingQueue, ScheduledExecutorService, TimeUnit} +import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} + +import gnu.trove.list.array.TLongArrayList +import gnu.trove.set.hash.TLongHashSet +import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} +import io.fabric8.kubernetes.client.KubernetesClient +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.KubernetesConf +import org.apache.spark.scheduler.ExecutorExited +import org.apache.spark.util.Utils + +private[spark] class ExecutorPodsEventHandler( + conf: SparkConf, + executorBuilder: KubernetesExecutorBuilder, + kubernetesClient: KubernetesClient, + eventProcessorExecutor: ScheduledExecutorService) { + + import ExecutorPodsEventHandler._ + + private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) + + private val totalExpectedExecutors = new AtomicInteger(0) + + private val eventQueue = new LinkedBlockingQueue[Seq[Pod]]() + + private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) + + private val kubernetesDriverPodName = conf + .get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse(throw new SparkException("Must specify the driver pod name")) + + private val driverPod = kubernetesClient.pods() + .withName(kubernetesDriverPodName) + .get() + + // Use sets of ids instead of counters to be able to handle duplicate events. + + // Executor IDs that have been requested from Kubernetes but are not running yet. + private val pendingExecutors = new TLongHashSet() + + // We could use CoarseGrainedSchedulerBackend#totalRegisteredExecutors here for tallying the + // executors that are running. But, here we choose instead to maintain all state within this + // class from the persecptive of the k8s API. Therefore whether or not this scheduler loop + // believes a scheduler is running is dictated by the K8s API rather than Spark's RPC events. + // We may need to consider where these perspectives may differ and which perspective should + // take precedence. + private val runningExecutors = new TLongHashSet() + + private var eventProcessorFuture: Future[_] = _ + + def start(applicationId: String, schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { + require(eventProcessorFuture == null, "Cannot start event processing twice.") + val eventProcessor = new Runnable { + override def run(): Unit = processEvents(applicationId, schedulerBackend) + } + eventProcessorFuture = eventProcessorExecutor.scheduleWithFixedDelay( + eventProcessor, 0L, 5L, TimeUnit.SECONDS) + } + + def stop(): Unit = { + if (eventProcessorFuture != null) { + eventProcessorFuture.cancel(true) + eventProcessorFuture = null + } + } + + private def processEvents( + applicationId: String, schedulerBackend: KubernetesClusterSchedulerBackend) { + val currentEvents = new java.util.ArrayList[Seq[Pod]](eventQueue.size()) + eventQueue.size()) + eventQueue.drainTo(currentEvents) + currentEvents.asScala.flatten.foreach { updatedPod => + val execId = updatedPod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL).toLong + val podPhase = updatedPod.getStatus.getPhase.toLowerCase + if (isDeleted(updatedPod)) { + removeExecutorFromSpark(schedulerBackend, updatedPod, execId) + } else { + updatedPod.getStatus.getPhase.toLowerCase match { + case "running" => + // If clause is for resililence to out of order operations - executor must be + // pending and first reach running. Without this check you may e.g. process a + // deletion event followed by some arbitrary modification event - we want the + // deletion event to "stick". + if (pendingExecutors.contains(execId)) { + pendingExecutors.remove(execId) + runningExecutors.add(execId) + } + // TODO (SPARK-24135) - handle more classes of errors + case "error" | "failed" | "succeeded" => + removeExecutorFromSpark(schedulerBackend, updatedPod, execId) + // If deletion failed on a previous try, we can try again if resync informs us the pod + // is still around. + // Delete as best attempt - duplicate deletes will throw an exception but the end state + // of getting rid of the pod is what matters. + if (!isDeleted(updatedPod)) { + Utils.tryLogNonFatalError { + kubernetesClient + .pods() + .withName(updatedPod.getMetadata.getName) + .delete() + } + } + } + } + } + + val currentRunningExecutors = runningExecutors.size + val currentTotalExpectedExecutors = totalExpectedExecutors.get + if (pendingExecutors.isEmpty && currentRunningExecutors < currentTotalExpectedExecutors) { + val numExecutorsToAllocate = math.min( + currentTotalExpectedExecutors - currentRunningExecutors, podAllocationSize) + val newExecutorIds = new TLongArrayList() + val podsToAllocate = mutable.Buffer.empty[Pod] + for (_ <- 0 until numExecutorsToAllocate) { + val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet() + val executorConf = KubernetesConf.createExecutorConf( + conf, + newExecutorId.toString, + applicationId, + driverPod) + val executorPod = executorBuilder.buildFromFeatures(executorConf) + val podWithAttachedContainer = new PodBuilder(executorPod.pod) + .editOrNewSpec() + .addToContainers(executorPod.container) + .endSpec() + .build() + newExecutorIds.add(newExecutorId) + } + kubernetesClient.pods().create(podsToAllocate: _*) + pendingExecutors.addAll(newExecutorIds) + } + } + + def sendUpdatedPodMetadata(updatedPod: Pod): Unit = { + eventQueue.add(Seq(updatedPod)) + } + + def sendUpdatedPodMetadata(updatedPods: Iterable[Pod]): Unit = { + eventQueue.add(updatedPods.toSeq) + } + + def setTotalExpectedExecutors(newTotal: Int): Unit = totalExpectedExecutors.set(newTotal) + + private def removeExecutorFromSpark( + schedulerBackend: KubernetesClusterSchedulerBackend, + updatedPod: Pod, + execId: Long): Unit = { + // Avoid removing twice from Spark's perspective. + if (pendingExecutors.contains(execId) || runningExecutors.contains(execId)) { + pendingExecutors.remove(execId) + runningExecutors.remove(execId) + val exitReason = findExitReason(updatedPod, execId) + schedulerBackend.doRemoveExecutor(execId.toString, exitReason) + } + } + + private def findExitReason(pod: Pod, execId: Long): ExecutorExited = { + val exitCode = findExitCode(pod) + val (exitCausedByApp, exitMessage) = if (isDeleted(pod)) { + (false, s"The executor with id $execId was deleted by a user or the framework.") + } else { + val msg = + s""" The executor with id $execId exited with exit code $exitCode. + | The API gave the following brief reason: ${pod.getStatus.getReason}. + | The API gave the following message: ${pod.getStatus.getMessage}. + | The API gave the following container statuses: + | ${pod.getStatus.getContainerStatuses.asScala.map(_.toString).mkString("\n===\n")} + """.stripMargin + (true, msg) + } + ExecutorExited(exitCode, exitCausedByApp, exitMessage) + } + + private def isDeleted(pod: Pod): Boolean = pod.getMetadata.getDeletionTimestamp != null + + private def findExitCode(pod: Pod): Int = { + pod.getStatus.getContainerStatuses.asScala.find { containerStatus => + containerStatus.getState.getTerminated != null + }.map { terminatedContainer => + terminatedContainer.getState.getTerminated.getExitCode.toInt + }.getOrElse(UNKNOWN_EXIT_CODE) + } +} + +private object ExecutorPodsEventHandler { + val UNKNOWN_EXIT_CODE = -1 +} + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala new file mode 100644 index 0000000000000..e5568816abd30 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.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.scheduler.cluster.k8s + +import java.util.concurrent.{Future, ScheduledExecutorService, TimeUnit} + +import io.fabric8.kubernetes.client.KubernetesClient +import scala.collection.JavaConverters._ + +import org.apache.spark.deploy.k8s.Constants._ + +private[spark] class ExecutorPodsPollingEventSource( + kubernetesClient: KubernetesClient, + eventHandler: ExecutorPodsEventHandler, + pollingExecutor: ScheduledExecutorService) { + + private var pollingFuture: Future[_] = _ + + def start(applicationId: String): Unit = { + require(pollingFuture == null, "Cannot start polling more than once.") + pollingFuture = pollingExecutor.scheduleWithFixedDelay( + new PollRunnable(applicationId), 0L, 30L, TimeUnit.SECONDS) + } + + def stop(): Unit = { + if (pollingFuture != null) { + pollingFuture.cancel(true) + pollingFuture = null + } + } + + private class PollRunnable(applicationId: String) extends Runnable { + override def run(): Unit = { + eventHandler.sendUpdatedPodMetadata(kubernetesClient + .pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .list() + .getItems + .asScala) + } + } + +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala new file mode 100644 index 0000000000000..767907e09ad42 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala @@ -0,0 +1,63 @@ +/* + * 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 io.fabric8.kubernetes.api.model.Pod +import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action + +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +private[spark] class ExecutorPodsWatchEventSource( + eventHandler: ExecutorPodsEventHandler, + kubernetesClient: KubernetesClient) extends Logging { + + private var watchConnection: Closeable = _ + + def start(applicationId: String): Unit = { + require(watchConnection == null, "Cannot start the watcher twice.") + watchConnection = kubernetesClient.pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId) + .withLabel(SPARK_POD_EXECUTOR_ROLE) + .watch(new ExecutorPodsWatcher()) + } + + def stop(): Unit = { + if (watchConnection != null) { + Utils.tryLogNonFatalError { + watchConnection.close() + } + watchConnection = null + } + } + + private class ExecutorPodsWatcher extends Watcher[Pod] { + override def eventReceived(action: Action, pod: Pod): Unit = { + eventHandler.sendUpdatedPodMetadata(pod) + } + + override def onClose(e: KubernetesClientException): Unit = { + logWarning("Kubernetes client has been closed (this is expected if the application is" + + " shutting down.)", e) + } + } + +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 0ea80dfbc0d97..710001ea02e50 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -56,17 +56,34 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) - val allocatorExecutor = ThreadUtils - .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( "kubernetes-executor-requests") + + val eventsProcessorExecutor = ThreadUtils + .newDaemonSingleThreadScheduledExecutor("kubernetes-executor-pods-event-handler") + val executorPodsEventHandler = new ExecutorPodsEventHandler( + sc.conf, + new KubernetesExecutorBuilder(), + kubernetesClient, + eventsProcessorExecutor) + + val podsWatchEventSource = new ExecutorPodsWatchEventSource( + executorPodsEventHandler, + kubernetesClient) + + val eventsPollingExecutor = ThreadUtils.newDaemonSingleThreadScheduledExecutor( + "kubernetes-executor-pod-polling-sync") + val podsPollingEventSource = new ExecutorPodsPollingEventSource( + kubernetesClient, executorPodsEventHandler, eventsPollingExecutor) + new KubernetesClusterSchedulerBackend( scheduler.asInstanceOf[TaskSchedulerImpl], sc.env.rpcEnv, - new KubernetesExecutorBuilder, kubernetesClient, - allocatorExecutor, - requestExecutorsService) + requestExecutorsService, + executorPodsEventHandler, + podsWatchEventSource, + podsPollingEventSource) } override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index d86664c81071b..87898418c2fdc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -16,60 +16,30 @@ */ 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 java.util.concurrent.ExecutorService -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 io.fabric8.kubernetes.client.KubernetesClient 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.deploy.k8s.KubernetesConf -import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv} -import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, TaskSchedulerImpl} +import org.apache.spark.rpc.{RpcAddress, RpcEnv} +import org.apache.spark.scheduler.{ExecutorLossReason, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SchedulerBackendUtils} import org.apache.spark.util.Utils private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, rpcEnv: RpcEnv, - executorBuilder: KubernetesExecutorBuilder, kubernetesClient: KubernetesClient, - allocatorExecutor: ScheduledExecutorService, - requestExecutorsService: ExecutorService) + requestExecutorsService: ExecutorService, + eventHandler: ExecutorPodsEventHandler, + watchEvents: ExecutorPodsWatchEventSource, + pollEvents: ExecutorPodsPollingEventSource) 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 @@ -77,372 +47,82 @@ private[spark] class KubernetesClusterSchedulerBackend( 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 executorLostReasonCheckMaxAttempts = conf.get( - KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS) - - 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 (_ <- 0 until math.min( - currentTotalExpectedExecutors - runningExecutorsToPods.size, podAllocationSize)) { - val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString - val executorConf = KubernetesConf.createExecutorConf( - conf, - executorId, - applicationId(), - driverPod) - val executorPod = executorBuilder.buildFromFeatures(executorConf) - val podWithAttachedContainer = new PodBuilder(executorPod.pod) - .editOrNewSpec() - .addToContainers(executorPod.container) - .endSpec() - .build() - - executorsToAllocate(executorId) = podWithAttachedContainer - 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 " and - // "kubectl describe pod ". 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 >= executorLostReasonCheckMaxAttempts) { - 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 + // Allow removeExecutor to be accessible by KubernetesSchedulerEventHandler + private[k8s] def doRemoveExecutor(executorId: String, reason: ExecutorLossReason): Unit = { + removeExecutor(executorId, reason) } 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.MILLISECONDS) - if (!Utils.isDynamicAllocationEnabled(conf)) { - doRequestTotalExecutors(initialExecutors) + eventHandler.setTotalExpectedExecutors(initialExecutors) } + eventHandler.start(applicationId(), this) + watchEvents.start(applicationId()) + pollEvents.start(applicationId()) } override def stop(): Unit = { - // stop allocation of new resources and caches. - allocatorExecutor.shutdown() - allocatorExecutor.awaitTermination(30, TimeUnit.SECONDS) - - // send stop message to executors so they shut down cleanly super.stop() - try { - val resource = executorWatchResource.getAndSet(null) - if (resource != null) { - resource.close() - } - } catch { - case e: Throwable => logWarning("Failed to close the executor pod watcher", e) + Utils.tryLogNonFatalError { + eventHandler.stop() } - // then delete the executor pods Utils.tryLogNonFatalError { - deleteExecutorPodsOnStop() - executorPodsByIPs.clear() + watchEvents.stop() } + Utils.tryLogNonFatalError { - logInfo("Closing kubernetes client") - kubernetesClient.close() + pollEvents.stop() } - } - /** - * @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 + Utils.tryLogNonFatalError { + kubernetesClient.pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId()) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .delete() } - 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 + Utils.tryLogNonFatalError { + kubernetesClient.close() } - 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: _*) + eventHandler.setTotalExpectedExecutors(requestedTotal) true } - private def deleteExecutorPodsOnStop(): Unit = { - val executorPodsToDelete = RUNNING_EXECUTOR_PODS_LOCK.synchronized { - val runningExecutorPodsCopy = Seq(runningExecutorsToPods.values.toSeq: _*) - runningExecutorsToPods.clear() - runningExecutorPodsCopy - } - kubernetesClient.pods().delete(executorPodsToDelete: _*) + override def sufficientResourcesRegistered(): Boolean = { + totalRegisteredExecutors.get() >= initialExecutors * minRegisteredRatio } - 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 doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { + kubernetesClient.pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId()) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .withLabelIn(SPARK_EXECUTOR_ID_LABEL, executorIds: _*) + .delete() + // Don't do anything else - let event handling from the Kubernetes API do the Spark changes } override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { new KubernetesDriverEndpoint(rpcEnv, properties) } - private class KubernetesDriverEndpoint( - rpcEnv: RpcEnv, - sparkProperties: Seq[(String, String)]) + 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) - } - } - } - } + // Don't do anything besides disabling the executor - allow the Kubernetes API events to + // drive the rest of the lifecycle decisions + // TODO what if we disconnect from a networking issue? Probably want to mark the executor + // to be deleted eventually. + addressToExecutorId.get(rpcAddress).foreach(disableExecutor) } } -} -private object KubernetesClusterSchedulerBackend { - private val UNKNOWN_EXIT_CODE = -1 } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala deleted file mode 100644 index 96065e83f069c..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ /dev/null @@ -1,449 +0,0 @@ -/* - * 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.{ContainerBuilder, 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.hamcrest.{BaseMatcher, Description, Matcher} -import org.mockito.{AdditionalAnswers, ArgumentCaptor, Matchers, 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.mockito.MockitoSugar._ -import scala.collection.JavaConverters._ -import scala.concurrent.Future - -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} -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 = "1m" - 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]] - - @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 executorBuilder: KubernetesExecutorBuilder = _ - - @Mock - private var kubernetesClient: KubernetesClient = _ - - @Mock - private var podOperations: PODS = _ - - @Mock - private var podsWithLabelOperations: LABELED_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 = _ - - @Mock - private var successFuture: Future[Boolean] = _ - - 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(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.key, 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(TimeUnit.MINUTES.toMillis(1)), - mockitoEq(TimeUnit.MILLISECONDS))).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) - - // Used by the CoarseGrainedSchedulerBackend when making RPC calls. - when(driverEndpointRef.ask[Boolean] - (any(classOf[Any])) - (any())).thenReturn(successFuture) - when(successFuture.failed).thenReturn(Future[Throwable] { - // emulate behavior of the Future.failed method. - throw new NoSuchElementException() - }(ThreadUtils.sameThread)) - } - - 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() - val firstResolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) - val secondResolvedPod = expectPodCreationWithId(2, SECOND_EXECUTOR_POD) - when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) - allocatorRunnable.getValue.run() - verify(podOperations).create(firstResolvedPod) - verify(podOperations).create(secondResolvedPod) - } - - 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() - val firstResolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) - val secondResolvedPod = 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(secondResolvedPod) - verify(podOperations, never()).delete(firstResolvedPod) - } - - 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()) - val firstResolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) - val secondResolvedPod = expectPodCreationWithId(2, SECOND_EXECUTOR_POD) - allocatorRunnable.getValue.run() - verify(podOperations).create(firstResolvedPod) - verify(podOperations, never()).create(secondResolvedPod) - 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(secondResolvedPod) - } - - test("Scaled down executors should be cleaned up") { - sparkConf - .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) - .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) - val scheduler = newSchedulerBackend() - scheduler.start() - - // The scheduler backend spins up one executor pod. - requestExecutorRunnable.getValue.run() - when(podOperations.create(any(classOf[Pod]))) - .thenAnswer(AdditionalAnswers.returnsFirstArg()) - val resolvedPod = 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) - - // Request that there are 0 executors and trigger deletion from driver. - scheduler.doRequestTotalExecutors(0) - requestExecutorRunnable.getAllValues.asScala.last.run() - scheduler.doKillExecutors(Seq("1")) - requestExecutorRunnable.getAllValues.asScala.last.run() - verify(podOperations, times(1)).delete(resolvedPod) - driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) - - val exitedPod = exitPod(resolvedPod, 0) - executorPodsWatcherArgument.getValue.eventReceived(Action.DELETED, exitedPod) - allocatorRunnable.getValue.run() - - // No more deletion attempts of the executors. - // This is graceful termination and should not be detected as a failure. - verify(podOperations, times(1)).delete(resolvedPod) - verify(driverEndpointRef, times(1)).send( - RemoveExecutor("1", ExecutorExited( - 0, - exitCausedByApp = false, - s"Container in pod ${exitedPod.getMetadata.getName} exited from" + - s" explicit termination request."))) - } - - test("Executors that fail should not be deleted.") { - sparkConf - .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) - .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) - - val scheduler = newSchedulerBackend() - scheduler.start() - val firstResolvedPod = 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(firstResolvedPod, 1)) - - // A replacement executor should be created but the error pod should persist. - val replacementPod = expectPodCreationWithId(2, SECOND_EXECUTOR_POD) - scheduler.doRequestTotalExecutors(1) - requestExecutorRunnable.getValue.run() - allocatorRunnable.getAllValues.asScala.last.run() - verify(podOperations, never()).delete(firstResolvedPod) - verify(driverEndpointRef).send( - RemoveExecutor("1", ExecutorExited( - 1, - exitCausedByApp = true, - s"Pod ${FIRST_EXECUTOR_POD.getMetadata.getName}'s executor container exited with" + - " exit status code 1."))) - } - - test("Executors disconnected due to unknown reasons are deleted and replaced.") { - sparkConf - .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) - .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) - val executorLostReasonCheckMaxAttempts = sparkConf.get( - KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS) - - val scheduler = newSchedulerBackend() - scheduler.start() - val firstResolvedPod = 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) - 1 to executorLostReasonCheckMaxAttempts foreach { _ => - allocatorRunnable.getValue.run() - verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) - } - - val recreatedResolvedPod = expectPodCreationWithId(2, SECOND_EXECUTOR_POD) - allocatorRunnable.getValue.run() - verify(podOperations).delete(firstResolvedPod) - verify(driverEndpointRef).send( - RemoveExecutor("1", SlaveLost("Executor lost for unknown reasons."))) - } - - test("Executors that fail to start on the Kubernetes API call rebuild in the next batch.") { - sparkConf - .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) - .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) - val scheduler = newSchedulerBackend() - scheduler.start() - val firstResolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) - when(podOperations.create(firstResolvedPod)) - .thenThrow(new RuntimeException("test")) - requestExecutorRunnable.getValue.run() - allocatorRunnable.getValue.run() - verify(podOperations, times(1)).create(firstResolvedPod) - val recreatedResolvedPod = expectPodCreationWithId(2, FIRST_EXECUTOR_POD) - allocatorRunnable.getValue.run() - verify(podOperations).create(recreatedResolvedPod) - } - - test("Executors that are initially created but the watch notices them fail are rebuilt" + - " in the next batch.") { - sparkConf - .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) - .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) - val scheduler = newSchedulerBackend() - scheduler.start() - val firstResolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) - when(podOperations.create(FIRST_EXECUTOR_POD)).thenAnswer(AdditionalAnswers.returnsFirstArg()) - requestExecutorRunnable.getValue.run() - allocatorRunnable.getValue.run() - verify(podOperations, times(1)).create(firstResolvedPod) - executorPodsWatcherArgument.getValue.eventReceived(Action.ERROR, firstResolvedPod) - val recreatedResolvedPod = expectPodCreationWithId(2, FIRST_EXECUTOR_POD) - allocatorRunnable.getValue.run() - verify(podOperations).create(recreatedResolvedPod) - } - - private def newSchedulerBackend(): KubernetesClusterSchedulerBackend = { - new KubernetesClusterSchedulerBackend( - taskSchedulerImpl, - rpcEnv, - executorBuilder, - kubernetesClient, - allocatorExecutor, - requestExecutorsService) { - - override def applicationId(): String = APP_ID - } - } - - private def exitPod(basePod: Pod, exitCode: Int): Pod = { - new PodBuilder(basePod) - .editStatus() - .addNewContainerStatus() - .withNewState() - .withNewTerminated() - .withExitCode(exitCode) - .endTerminated() - .endState() - .endContainerStatus() - .endStatus() - .build() - } - - private def expectPodCreationWithId(executorId: Int, expectedPod: Pod): Pod = { - val resolvedPod = new PodBuilder(expectedPod) - .editMetadata() - .addToLabels(SPARK_EXECUTOR_ID_LABEL, executorId.toString) - .endMetadata() - .build() - val resolvedContainer = new ContainerBuilder().build() - when(executorBuilder.buildFromFeatures(Matchers.argThat( - new BaseMatcher[KubernetesConf[KubernetesExecutorSpecificConf]] { - override def matches(argument: scala.Any) - : Boolean = { - argument.isInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]] && - argument.asInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]] - .roleSpecificConf.executorId == executorId.toString - } - - override def describeTo(description: Description): Unit = {} - }))).thenReturn(SparkPod(resolvedPod, resolvedContainer)) - new PodBuilder(resolvedPod) - .editSpec() - .addToContainers(resolvedContainer) - .endSpec() - .build() - } -} From 60990f1c4dc1169ec9cf3f74a2b623805eb4a627 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 18 May 2018 15:01:03 -0700 Subject: [PATCH 02/47] Fix build --- .../spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala index 40712a9c2a2fa..77695080cca8f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala @@ -65,7 +65,7 @@ private[spark] class ExecutorPodsEventHandler( // We could use CoarseGrainedSchedulerBackend#totalRegisteredExecutors here for tallying the // executors that are running. But, here we choose instead to maintain all state within this // class from the persecptive of the k8s API. Therefore whether or not this scheduler loop - // believes a scheduler is running is dictated by the K8s API rather than Spark's RPC events. + // believes an executor is running is dictated by the K8s API rather than Spark's RPC events. // We may need to consider where these perspectives may differ and which perspective should // take precedence. private val runningExecutors = new TLongHashSet() @@ -91,7 +91,6 @@ private[spark] class ExecutorPodsEventHandler( private def processEvents( applicationId: String, schedulerBackend: KubernetesClusterSchedulerBackend) { val currentEvents = new java.util.ArrayList[Seq[Pod]](eventQueue.size()) - eventQueue.size()) eventQueue.drainTo(currentEvents) currentEvents.asScala.flatten.foreach { updatedPod => val execId = updatedPod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL).toLong From f3bb80aa47a69fbf1727abb86a93ef981a4483d6 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 18 May 2018 15:42:05 -0700 Subject: [PATCH 03/47] Dependencies --- dev/deps/spark-deps-hadoop-2.6 | 1 + dev/deps/spark-deps-hadoop-2.7 | 1 + dev/deps/spark-deps-hadoop-3.1 | 1 + 3 files changed, 3 insertions(+) diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index e710e26348117..2e1ed240c1a64 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -190,6 +190,7 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar +trove4j-3.0.1.jar univocity-parsers-2.6.3.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 97ad17a9ff7b1..c914a6f30eb71 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -191,6 +191,7 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar +trove4j-3.0.1.jar univocity-parsers-2.6.3.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index e21bfef8c4291..04aaf0878c1ed 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -211,6 +211,7 @@ stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar token-provider-1.0.1.jar +trove4j-3.0.1.jar univocity-parsers-2.6.3.jar validation-api-1.1.0.Final.jar woodstox-core-5.0.3.jar From 3343ba6a7a355594a677e186c174e5772cba538e Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 18 May 2018 16:06:04 -0700 Subject: [PATCH 04/47] Adding some logging. --- .../cluster/k8s/ExecutorPodsEventHandler.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala index 77695080cca8f..babed78fc6167 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala @@ -30,6 +30,7 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesConf +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.ExecutorExited import org.apache.spark.util.Utils @@ -37,7 +38,7 @@ private[spark] class ExecutorPodsEventHandler( conf: SparkConf, executorBuilder: KubernetesExecutorBuilder, kubernetesClient: KubernetesClient, - eventProcessorExecutor: ScheduledExecutorService) { + eventProcessorExecutor: ScheduledExecutorService) extends Logging { import ExecutorPodsEventHandler._ @@ -132,9 +133,10 @@ private[spark] class ExecutorPodsEventHandler( if (pendingExecutors.isEmpty && currentRunningExecutors < currentTotalExpectedExecutors) { val numExecutorsToAllocate = math.min( currentTotalExpectedExecutors - currentRunningExecutors, podAllocationSize) + logInfo(s"Going to request $numExecutorsToAllocate executors from Kubernetes.") val newExecutorIds = new TLongArrayList() val podsToAllocate = mutable.Buffer.empty[Pod] - for (_ <- 0 until numExecutorsToAllocate) { + for ( _ <- 0 until numExecutorsToAllocate) { val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet() val executorConf = KubernetesConf.createExecutorConf( conf, @@ -151,6 +153,12 @@ private[spark] class ExecutorPodsEventHandler( } kubernetesClient.pods().create(podsToAllocate: _*) pendingExecutors.addAll(newExecutorIds) + } else if (currentRunningExecutors == currentTotalExpectedExecutors) { + logDebug("Current number of running executors is equal to the number of requested executors." + + " Not scaling up further.") + } else if (!pendingExecutors.isEmpty) { + logInfo(s"Still waiting for ${pendingExecutors.size} executors to begin running before" + + s" requesting for more executors.") } } From 30b7f170175a0eef643cdfa1373bf9671cb75ade Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 18 May 2018 16:17:50 -0700 Subject: [PATCH 05/47] Specifically initialize things with null. More logs. --- .../scheduler/cluster/k8s/ExecutorPodsEventHandler.scala | 4 +++- .../cluster/k8s/ExecutorPodsPollingEventSource.scala | 2 +- .../scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala index babed78fc6167..6d197e8067580 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala @@ -71,10 +71,12 @@ private[spark] class ExecutorPodsEventHandler( // take precedence. private val runningExecutors = new TLongHashSet() - private var eventProcessorFuture: Future[_] = _ + private var eventProcessorFuture: Future[_] = null def start(applicationId: String, schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { require(eventProcessorFuture == null, "Cannot start event processing twice.") + logInfo(s"Starting Kubernetes executor pods event handler for application with" + + s" id $applicationId.") val eventProcessor = new Runnable { override def run(): Unit = processEvents(applicationId, schedulerBackend) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala index e5568816abd30..05462fc89ee7b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala @@ -28,7 +28,7 @@ private[spark] class ExecutorPodsPollingEventSource( eventHandler: ExecutorPodsEventHandler, pollingExecutor: ScheduledExecutorService) { - private var pollingFuture: Future[_] = _ + private var pollingFuture: Future[_] = null def start(applicationId: String): Unit = { require(pollingFuture == null, "Cannot start polling more than once.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala index 767907e09ad42..48e4c61d3a600 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala @@ -30,7 +30,7 @@ private[spark] class ExecutorPodsWatchEventSource( eventHandler: ExecutorPodsEventHandler, kubernetesClient: KubernetesClient) extends Logging { - private var watchConnection: Closeable = _ + private var watchConnection: Closeable = null def start(applicationId: String): Unit = { require(watchConnection == null, "Cannot start the watcher twice.") From 522b0799b31f064e378c688eeecec3eca3b0810d Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 18 May 2018 16:20:23 -0700 Subject: [PATCH 06/47] Fix scalastyle --- .../scheduler/cluster/k8s/ExecutorPodsEventHandler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala index 6d197e8067580..0d63d5fcd1360 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala @@ -156,8 +156,8 @@ private[spark] class ExecutorPodsEventHandler( kubernetesClient.pods().create(podsToAllocate: _*) pendingExecutors.addAll(newExecutorIds) } else if (currentRunningExecutors == currentTotalExpectedExecutors) { - logDebug("Current number of running executors is equal to the number of requested executors." + - " Not scaling up further.") + logDebug("Current number of running executors is equal to the number of requested" + + " executors. Not scaling up further.") } else if (!pendingExecutors.isEmpty) { logInfo(s"Still waiting for ${pendingExecutors.size} executors to begin running before" + s" requesting for more executors.") From 600e25f0ca70eddd5407d76c1210b9b637eeae77 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 18 May 2018 16:38:52 -0700 Subject: [PATCH 07/47] Actually create the pods --- .../spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala index 0d63d5fcd1360..ddbbf482926da 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala @@ -151,6 +151,7 @@ private[spark] class ExecutorPodsEventHandler( .addToContainers(executorPod.container) .endSpec() .build() + podsToAllocate.add(podWithAttachedContainer) newExecutorIds.add(newExecutorId) } kubernetesClient.pods().create(podsToAllocate: _*) From 931529a74c07b3212dcbe34e7329384555777146 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 18 May 2018 16:46:45 -0700 Subject: [PATCH 08/47] Fix build --- .../spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala index ddbbf482926da..fd4bc7d913f7b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala @@ -151,7 +151,7 @@ private[spark] class ExecutorPodsEventHandler( .addToContainers(executorPod.container) .endSpec() .build() - podsToAllocate.add(podWithAttachedContainer) + podsToAllocate += podWithAttachedContainer newExecutorIds.add(newExecutorId) } kubernetesClient.pods().create(podsToAllocate: _*) From 9e5abfbf2d21b788e26315f11cfeb1a83c82262e Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 21 May 2018 14:55:30 -0700 Subject: [PATCH 09/47] Request only one pod at a time. Use logNonFatalError to log exceptions. --- .../cluster/k8s/ExecutorPodsEventHandler.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala index fd4bc7d913f7b..4c7bdbf0ebd6a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala @@ -78,7 +78,11 @@ private[spark] class ExecutorPodsEventHandler( logInfo(s"Starting Kubernetes executor pods event handler for application with" + s" id $applicationId.") val eventProcessor = new Runnable { - override def run(): Unit = processEvents(applicationId, schedulerBackend) + override def run(): Unit = { + Utils.tryLogNonFatalError { + processEvents(applicationId, schedulerBackend) + } + } } eventProcessorFuture = eventProcessorExecutor.scheduleWithFixedDelay( eventProcessor, 0L, 5L, TimeUnit.SECONDS) @@ -113,7 +117,6 @@ private[spark] class ExecutorPodsEventHandler( } // TODO (SPARK-24135) - handle more classes of errors case "error" | "failed" | "succeeded" => - removeExecutorFromSpark(schedulerBackend, updatedPod, execId) // If deletion failed on a previous try, we can try again if resync informs us the pod // is still around. // Delete as best attempt - duplicate deletes will throw an exception but the end state @@ -126,6 +129,7 @@ private[spark] class ExecutorPodsEventHandler( .delete() } } + removeExecutorFromSpark(schedulerBackend, updatedPod, execId) } } } @@ -151,11 +155,9 @@ private[spark] class ExecutorPodsEventHandler( .addToContainers(executorPod.container) .endSpec() .build() - podsToAllocate += podWithAttachedContainer - newExecutorIds.add(newExecutorId) + kubernetesClient.pods().create(podWithAttachedContainer) + pendingExecutors.add(newExecutorId) } - kubernetesClient.pods().create(podsToAllocate: _*) - pendingExecutors.addAll(newExecutorIds) } else if (currentRunningExecutors == currentTotalExpectedExecutors) { logDebug("Current number of running executors is equal to the number of requested" + " executors. Not scaling up further.") From 2156a20f0d64fe34923674cee6a69c4387debac4 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 21 May 2018 16:44:23 -0700 Subject: [PATCH 10/47] Initial tests --- pom.xml | 6 + resource-managers/kubernetes/core/pom.xml | 12 +- .../k8s/ExecutorPodsEventHandler.scala | 4 +- .../k8s/ExecutorPodsEventHandlerSuite.scala | 179 ++++++++++++++++++ 4 files changed, 197 insertions(+), 4 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala diff --git a/pom.xml b/pom.xml index 43054371e0ad5..7723b848a700f 100644 --- a/pom.xml +++ b/pom.xml @@ -766,6 +766,12 @@ 1.10.19 test + + org.jmock + jmock-junit4 + test + 2.8.4 + org.scalacheck scalacheck_${scala.binary.version} diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 79a8d9b0524d8..f314990d529ad 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -83,6 +83,12 @@ + + com.squareup.okhttp3 + okhttp + 3.8.1 + + org.mockito mockito-core @@ -90,9 +96,9 @@ - com.squareup.okhttp3 - okhttp - 3.8.1 + org.jmock + jmock-junit4 + test diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala index 4c7bdbf0ebd6a..e40d26652bc07 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala @@ -50,6 +50,8 @@ private[spark] class ExecutorPodsEventHandler( private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) + private val podAllocationDelay = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) + private val kubernetesDriverPodName = conf .get(KUBERNETES_DRIVER_POD_NAME) .getOrElse(throw new SparkException("Must specify the driver pod name")) @@ -85,7 +87,7 @@ private[spark] class ExecutorPodsEventHandler( } } eventProcessorFuture = eventProcessorExecutor.scheduleWithFixedDelay( - eventProcessor, 0L, 5L, TimeUnit.SECONDS) + eventProcessor, 0L, podAllocationDelay, TimeUnit.MILLISECONDS) } def stop(): Unit = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala new file mode 100644 index 0000000000000..f516d8c41103d --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala @@ -0,0 +1,179 @@ +/* + * 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.TimeUnit + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, DoneablePod, Pod, PodBuilder, PodList} +import io.fabric8.kubernetes.client.KubernetesClient +import io.fabric8.kubernetes.client.dsl.{MixedOperation, PodResource} +import org.jmock.lib.concurrent.DeterministicScheduler +import org.mockito.{ArgumentMatcher, Matchers, MockitoAnnotations} +import org.mockito.Mockito.{never, verify, when} +import org.mockito.MockitoAnnotations.Mock +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ + +class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { + + private type Pods = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + + private val driverPodName = "driver" + + private val appId = "spark" + + private val driverPod = new PodBuilder() + .withNewMetadata() + .withName(driverPodName) + .addToLabels(SPARK_APP_ID_LABEL, appId) + .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_DRIVER_ROLE) + .withUid("driver-pod-uid") + .endMetadata() + .build() + + private val conf = new SparkConf().set(KUBERNETES_DRIVER_POD_NAME, driverPodName) + + private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) + + private val podAllocationDelay = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) + + private val eventProcessorExecutor = new DeterministicScheduler + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var podOperations: Pods = _ + + @Mock + private var namedPods: PodResource[Pod, DoneablePod] = _ + + @Mock + private var executorBuilder: KubernetesExecutorBuilder = _ + + @Mock + private var schedulerBackend: KubernetesClusterSchedulerBackend = _ + + private var eventHandlerUnderTest: ExecutorPodsEventHandler = _ + + before { + MockitoAnnotations.initMocks(this) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withName(driverPodName)).thenReturn(namedPods) + when(namedPods.get).thenReturn(driverPod) + when(executorBuilder.buildFromFeatures(kubernetesConfWithCorrectFields())) + .thenAnswer(executorPodAnswer()) + eventHandlerUnderTest = new ExecutorPodsEventHandler( + conf, executorBuilder, kubernetesClient, eventProcessorExecutor) + eventHandlerUnderTest.start(appId, schedulerBackend) + } + + test("Initially request executors in batches. Do not request another batch if the" + + " first has not finished.") { + eventHandlerUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) + eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) + for (nextId <- 1 to podAllocationSize) { + verify(podOperations).create(podWithAttachedContainerForId(nextId)) + } + verify(podOperations, never()).create( + podWithAttachedContainerForId(podAllocationSize + 1)) + } + + test("Request executors in batches. Allow another batch to be requested if" + + " some start running.") { + eventHandlerUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) + eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) + for (execId <- 1 to podAllocationSize) { + eventHandlerUnderTest.sendUpdatedPodMetadata(runExecutor(execId)) + } + eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) + verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) + verify(podOperations, never()).create(podWithAttachedContainerForId(podAllocationSize + 2)) + } + + private def runExecutor(executorId: Int): Pod = { + new PodBuilder(podWithAttachedContainerForId(executorId)) + .editOrNewStatus() + .withPhase("running") + .endStatus() + .build() + } + + private def kubernetesConfWithCorrectFields(): KubernetesConf[KubernetesExecutorSpecificConf] = + Matchers.argThat(new ArgumentMatcher[KubernetesConf[KubernetesExecutorSpecificConf]] { + override def matches(argument: scala.Any): Boolean = { + if (!argument.isInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]]) { + false + } else { + val k8sConf = argument.asInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]] + val executorSpecificConf = k8sConf.roleSpecificConf + val expectedK8sConf = KubernetesConf.createExecutorConf( + conf, + executorSpecificConf.executorId, + appId, + driverPod) + k8sConf.sparkConf.getAll.toMap == conf.getAll.toMap && + // Since KubernetesConf.createExecutorConf clones the SparkConf object, force + // deep equality comparison for the SparkConf object and use object equality + // comparison on all other fields. + k8sConf.copy(sparkConf = conf) == expectedK8sConf.copy(sparkConf = conf) + } + } + }) + + private def executorPodAnswer(): Answer[SparkPod] = { + new Answer[SparkPod] { + override def answer(invocation: InvocationOnMock): SparkPod = { + val k8sConf = invocation.getArgumentAt( + 0, classOf[KubernetesConf[KubernetesExecutorSpecificConf]]) + executorPodWithId(k8sConf.roleSpecificConf.executorId) + } + } + } + + private def podWithAttachedContainerForId(executorId: Int): Pod = { + val sparkPod = executorPodWithId(executorId.toString) + val podWithAttachedContainer = new PodBuilder(sparkPod.pod) + .editOrNewSpec() + .addToContainers(sparkPod.container) + .endSpec() + .build() + podWithAttachedContainer + } + + private def executorPodWithId(executorId: String): SparkPod = { + val pod = new PodBuilder() + .withNewMetadata() + .withName(s"spark-executor-$executorId") + .addToLabels(SPARK_APP_ID_LABEL, appId) + .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .addToLabels(SPARK_EXECUTOR_ID_LABEL, executorId) + .endMetadata() + .build() + val container = new ContainerBuilder() + .withName("spark-executor") + .withImage("k8s-spark") + .build() + SparkPod(pod, container) + } +} From aabc1872280f2f1c993a619e489c70370144990f Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 21 May 2018 16:47:51 -0700 Subject: [PATCH 11/47] Don't use GNU Trove --- pom.xml | 6 ------ resource-managers/kubernetes/core/pom.xml | 6 ------ .../cluster/k8s/ExecutorPodsEventHandler.scala | 14 ++++++-------- .../k8s/ExecutorPodsEventHandlerSuite.scala | 8 ++++++-- 4 files changed, 12 insertions(+), 22 deletions(-) diff --git a/pom.xml b/pom.xml index 7723b848a700f..6fbf58dec9695 100644 --- a/pom.xml +++ b/pom.xml @@ -150,7 +150,6 @@ 4.5.4 4.4.8 - 3.0.1 3.1 3.4.1 @@ -435,11 +434,6 @@ commons-collections ${commons.collections.version} - - net.sf.trove4j - trove4j - ${trove.version} - org.apache.ivy ivy diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index f314990d529ad..a6dd47a6b7d95 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -70,12 +70,6 @@ ${fasterxml.jackson.version} - - net.sf.trove4j - trove4j - ${trove.version} - - com.google.guava diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala index e40d26652bc07..06fa900d7f6b3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala @@ -19,8 +19,6 @@ package org.apache.spark.scheduler.cluster.k8s import java.util.concurrent.{Future, LinkedBlockingQueue, ScheduledExecutorService, TimeUnit} import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} -import gnu.trove.list.array.TLongArrayList -import gnu.trove.set.hash.TLongHashSet import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.JavaConverters._ @@ -63,7 +61,7 @@ private[spark] class ExecutorPodsEventHandler( // Use sets of ids instead of counters to be able to handle duplicate events. // Executor IDs that have been requested from Kubernetes but are not running yet. - private val pendingExecutors = new TLongHashSet() + private val pendingExecutors = mutable.Set.empty[Long] // We could use CoarseGrainedSchedulerBackend#totalRegisteredExecutors here for tallying the // executors that are running. But, here we choose instead to maintain all state within this @@ -71,9 +69,9 @@ private[spark] class ExecutorPodsEventHandler( // believes an executor is running is dictated by the K8s API rather than Spark's RPC events. // We may need to consider where these perspectives may differ and which perspective should // take precedence. - private val runningExecutors = new TLongHashSet() + private val runningExecutors = mutable.Set.empty[Long] - private var eventProcessorFuture: Future[_] = null + private var eventProcessorFuture: Future[_] = _ def start(applicationId: String, schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { require(eventProcessorFuture == null, "Cannot start event processing twice.") @@ -142,7 +140,7 @@ private[spark] class ExecutorPodsEventHandler( val numExecutorsToAllocate = math.min( currentTotalExpectedExecutors - currentRunningExecutors, podAllocationSize) logInfo(s"Going to request $numExecutorsToAllocate executors from Kubernetes.") - val newExecutorIds = new TLongArrayList() + val newExecutorIds = mutable.Buffer.empty[Long] val podsToAllocate = mutable.Buffer.empty[Pod] for ( _ <- 0 until numExecutorsToAllocate) { val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet() @@ -158,12 +156,12 @@ private[spark] class ExecutorPodsEventHandler( .endSpec() .build() kubernetesClient.pods().create(podWithAttachedContainer) - pendingExecutors.add(newExecutorId) + pendingExecutors += newExecutorId } } else if (currentRunningExecutors == currentTotalExpectedExecutors) { logDebug("Current number of running executors is equal to the number of requested" + " executors. Not scaling up further.") - } else if (!pendingExecutors.isEmpty) { + } else if (pendingExecutors.nonEmpty) { logInfo(s"Still waiting for ${pendingExecutors.size} executors to begin running before" + s" requesting for more executors.") } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala index f516d8c41103d..555ca426484d0 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala @@ -100,13 +100,17 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { } test("Request executors in batches. Allow another batch to be requested if" + - " some start running.") { + " all pending executors start running.") { eventHandlerUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) - for (execId <- 1 to podAllocationSize) { + for (execId <- 1 until podAllocationSize) { eventHandlerUnderTest.sendUpdatedPodMetadata(runExecutor(execId)) } eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) + verify(podOperations, never()).create( + podWithAttachedContainerForId(podAllocationSize + 1)) + eventHandlerUnderTest.sendUpdatedPodMetadata(runExecutor(podAllocationSize)) + eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) verify(podOperations, never()).create(podWithAttachedContainerForId(podAllocationSize + 2)) } From ee0d196045e794f85dc8e4d56ec50a87242a0f1b Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 21 May 2018 17:28:17 -0700 Subject: [PATCH 12/47] Add another test. --- .../k8s/ExecutorPodsEventHandler.scala | 2 +- .../k8s/ExecutorPodsEventHandlerSuite.scala | 89 ++++++++++++++++--- 2 files changed, 78 insertions(+), 13 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala index 06fa900d7f6b3..e172b15df6598 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala @@ -196,7 +196,7 @@ private[spark] class ExecutorPodsEventHandler( (false, s"The executor with id $execId was deleted by a user or the framework.") } else { val msg = - s""" The executor with id $execId exited with exit code $exitCode. + s"""The executor with id $execId exited with exit code $exitCode. | The API gave the following brief reason: ${pod.getStatus.getReason}. | The API gave the following message: ${pod.getStatus.getMessage}. | The API gave the following container statuses: diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala index 555ca426484d0..c364e9ddc29a8 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala @@ -23,16 +23,20 @@ import io.fabric8.kubernetes.client.KubernetesClient import io.fabric8.kubernetes.client.dsl.{MixedOperation, PodResource} import org.jmock.lib.concurrent.DeterministicScheduler import org.mockito.{ArgumentMatcher, Matchers, MockitoAnnotations} -import org.mockito.Mockito.{never, verify, when} +import org.mockito.Matchers.any +import org.mockito.Mockito.{mock, never, times, verify, when} import org.mockito.MockitoAnnotations.Mock import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ +import scala.collection.mutable import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.scheduler.ExecutorExited class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { @@ -59,6 +63,8 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { private val eventProcessorExecutor = new DeterministicScheduler + private var namedExecutorPods: mutable.Map[String, PodResource[Pod, DoneablePod]] = _ + @Mock private var kubernetesClient: KubernetesClient = _ @@ -66,7 +72,7 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { private var podOperations: Pods = _ @Mock - private var namedPods: PodResource[Pod, DoneablePod] = _ + private var driverPodOperations: PodResource[Pod, DoneablePod] = _ @Mock private var executorBuilder: KubernetesExecutorBuilder = _ @@ -78,9 +84,11 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { before { MockitoAnnotations.initMocks(this) + namedExecutorPods = mutable.Map.empty[String, PodResource[Pod, DoneablePod]] when(kubernetesClient.pods()).thenReturn(podOperations) - when(podOperations.withName(driverPodName)).thenReturn(namedPods) - when(namedPods.get).thenReturn(driverPod) + when(podOperations.withName(any(classOf[String]))).thenAnswer(namedPodsAnswer()) + when(podOperations.withName(driverPodName)).thenReturn(driverPodOperations) + when(driverPodOperations.get).thenReturn(driverPod) when(executorBuilder.buildFromFeatures(kubernetesConfWithCorrectFields())) .thenAnswer(executorPodAnswer()) eventHandlerUnderTest = new ExecutorPodsEventHandler( @@ -91,7 +99,7 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { test("Initially request executors in batches. Do not request another batch if the" + " first has not finished.") { eventHandlerUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) - eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) + runProcessor() for (nextId <- 1 to podAllocationSize) { verify(podOperations).create(podWithAttachedContainerForId(nextId)) } @@ -102,20 +110,47 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { test("Request executors in batches. Allow another batch to be requested if" + " all pending executors start running.") { eventHandlerUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) - eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) + runProcessor() for (execId <- 1 until podAllocationSize) { - eventHandlerUnderTest.sendUpdatedPodMetadata(runExecutor(execId)) + eventHandlerUnderTest.sendUpdatedPodMetadata(runningExecutor(execId)) } - eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) + runProcessor() verify(podOperations, never()).create( podWithAttachedContainerForId(podAllocationSize + 1)) - eventHandlerUnderTest.sendUpdatedPodMetadata(runExecutor(podAllocationSize)) - eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) + eventHandlerUnderTest.sendUpdatedPodMetadata(runningExecutor(podAllocationSize)) + runProcessor() + verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) + runProcessor() + verify(podOperations, times(podAllocationSize + 1)).create(any(classOf[Pod])) + } + + test("When a current batch reaches error states immediately, re-request" + + " them on the next batch.") { + eventHandlerUnderTest.setTotalExpectedExecutors(podAllocationSize) + runProcessor() + for (execId <- 1 until podAllocationSize) { + eventHandlerUnderTest.sendUpdatedPodMetadata(runningExecutor(execId)) + } + val failedPod = failedExecutorWithoutDeletion(podAllocationSize) + eventHandlerUnderTest.sendUpdatedPodMetadata(failedPod) + runProcessor() + val msg = s"""The executor with id $podAllocationSize exited with exit code 1. + | The API gave the following brief reason: ${failedPod.getStatus.getReason}. + | The API gave the following message: ${failedPod.getStatus.getMessage}. + | The API gave the following container statuses: + | ${failedPod.getStatus.getContainerStatuses.asScala.map(_.toString).mkString("\n===\n")} + """.stripMargin + val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) + verify(schedulerBackend).doRemoveExecutor(podAllocationSize.toString, expectedLossReason) verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) - verify(podOperations, never()).create(podWithAttachedContainerForId(podAllocationSize + 2)) + verify(namedExecutorPods(failedPod.getMetadata.getName)).delete() } - private def runExecutor(executorId: Int): Pod = { + private def runProcessor(): Unit = { + eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) + } + + private def runningExecutor(executorId: Int): Pod = { new PodBuilder(podWithAttachedContainerForId(executorId)) .editOrNewStatus() .withPhase("running") @@ -123,6 +158,26 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { .build() } + private def failedExecutorWithoutDeletion(executorId: Int): Pod = { + new PodBuilder(podWithAttachedContainerForId(executorId)) + .editOrNewStatus() + .withPhase("error") + .addNewContainerStatus() + .withName("spark-executor") + .withImage("k8s-spark") + .withNewState() + .withNewTerminated() + .withMessage("Failed") + .withExitCode(1) + .endTerminated() + .endState() + .endContainerStatus() + .withMessage("Executor failed.") + .withReason("Executor failed because of a thrown error.") + .endStatus() + .build() + } + private def kubernetesConfWithCorrectFields(): KubernetesConf[KubernetesExecutorSpecificConf] = Matchers.argThat(new ArgumentMatcher[KubernetesConf[KubernetesExecutorSpecificConf]] { override def matches(argument: scala.Any): Boolean = { @@ -155,6 +210,16 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { } } + private def namedPodsAnswer(): Answer[PodResource[Pod, DoneablePod]] = { + new Answer[PodResource[Pod, DoneablePod]] { + override def answer(invocation: InvocationOnMock): PodResource[Pod, DoneablePod] = { + val podName = invocation.getArgumentAt(0, classOf[String]) + namedExecutorPods.getOrElseUpdate( + podName, mock(classOf[PodResource[Pod, DoneablePod]])) + } + } + } + private def podWithAttachedContainerForId(executorId: Int): Pod = { val sparkPod = executorPodWithId(executorId.toString) val podWithAttachedContainer = new PodBuilder(sparkPod.pod) From c2b973320840aadac7eaaa59f71d93284e0dfbbb Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 21 May 2018 17:31:26 -0700 Subject: [PATCH 13/47] Fix dependencies --- dev/deps/spark-deps-hadoop-2.6 | 1 - dev/deps/spark-deps-hadoop-2.7 | 1 - dev/deps/spark-deps-hadoop-3.1 | 1 - 3 files changed, 3 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 2e1ed240c1a64..e710e26348117 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -190,7 +190,6 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -trove4j-3.0.1.jar univocity-parsers-2.6.3.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index c914a6f30eb71..97ad17a9ff7b1 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -191,7 +191,6 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -trove4j-3.0.1.jar univocity-parsers-2.6.3.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index 04aaf0878c1ed..e21bfef8c4291 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -211,7 +211,6 @@ stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar token-provider-1.0.1.jar -trove4j-3.0.1.jar univocity-parsers-2.6.3.jar validation-api-1.1.0.Final.jar woodstox-core-5.0.3.jar From caffe23d8996d55f4ebd7cd0693ef82c21264ef9 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 21 May 2018 18:59:40 -0700 Subject: [PATCH 14/47] More tests --- .../k8s/ExecutorPodsEventHandler.scala | 19 +++--- .../k8s/ExecutorPodsEventHandlerSuite.scala | 62 +++++++++++++++---- 2 files changed, 61 insertions(+), 20 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala index e172b15df6598..3b5e9a3fe04ee 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala @@ -195,18 +195,23 @@ private[spark] class ExecutorPodsEventHandler( val (exitCausedByApp, exitMessage) = if (isDeleted(pod)) { (false, s"The executor with id $execId was deleted by a user or the framework.") } else { - val msg = - s"""The executor with id $execId exited with exit code $exitCode. - | The API gave the following brief reason: ${pod.getStatus.getReason}. - | The API gave the following message: ${pod.getStatus.getMessage}. - | The API gave the following container statuses: - | ${pod.getStatus.getContainerStatuses.asScala.map(_.toString).mkString("\n===\n")} - """.stripMargin + val msg = exitReasonMessage(pod, execId, exitCode) (true, msg) } ExecutorExited(exitCode, exitCausedByApp, exitMessage) } + private def exitReasonMessage(pod: Pod, execId: Long, exitCode: Int) = { + s""" + |The executor with id $execId exited with exit code $exitCode. + |The API gave the following brief reason: ${pod.getStatus.getReason} + |The API gave the following message: ${pod.getStatus.getMessage} + |The API gave the following container statuses: + | + |${pod.getStatus.getContainerStatuses.asScala.map(_.toString).mkString("\n===\n")} + """.stripMargin + } + private def isDeleted(pod: Pod): Boolean = pod.getMetadata.getDeletionTimestamp != null private def findExitCode(pod: Pod): Int = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala index c364e9ddc29a8..40eb5993ead09 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala @@ -134,18 +134,44 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { val failedPod = failedExecutorWithoutDeletion(podAllocationSize) eventHandlerUnderTest.sendUpdatedPodMetadata(failedPod) runProcessor() - val msg = s"""The executor with id $podAllocationSize exited with exit code 1. - | The API gave the following brief reason: ${failedPod.getStatus.getReason}. - | The API gave the following message: ${failedPod.getStatus.getMessage}. - | The API gave the following container statuses: - | ${failedPod.getStatus.getContainerStatuses.asScala.map(_.toString).mkString("\n===\n")} - """.stripMargin + val msg = exitReasonMessage(podAllocationSize, failedPod) val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) verify(schedulerBackend).doRemoveExecutor(podAllocationSize.toString, expectedLossReason) verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) verify(namedExecutorPods(failedPod.getMetadata.getName)).delete() } + test("When a current batch reaches a running state and then one executor reaches an error" + + " state, re-request it on the next batch.") { + eventHandlerUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) + runProcessor() + for (execId <- 1 to podAllocationSize) { + eventHandlerUnderTest.sendUpdatedPodMetadata(runningExecutor(execId)) + } + runProcessor() + eventHandlerUnderTest.sendUpdatedPodMetadata(runningExecutor(podAllocationSize + 1)) + val failedExecutorId = podAllocationSize - 1 + val failedPod = failedExecutorWithoutDeletion(failedExecutorId) + eventHandlerUnderTest.sendUpdatedPodMetadata(failedPod) + runProcessor() + val msg = exitReasonMessage(failedExecutorId, failedPod) + val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) + verify(schedulerBackend).doRemoveExecutor(failedExecutorId.toString, expectedLossReason) + verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 2)) + verify(namedExecutorPods(failedPod.getMetadata.getName)).delete() + } + + private def exitReasonMessage(failedExecutorId: Int, failedPod: Pod): String = { + s""" + |The executor with id $failedExecutorId exited with exit code 1. + |The API gave the following brief reason: ${failedPod.getStatus.getReason} + |The API gave the following message: ${failedPod.getStatus.getMessage} + |The API gave the following container statuses: + | + |${failedPod.getStatus.getContainerStatuses.asScala.map(_.toString).mkString("\n===\n")} + """.stripMargin + } + private def runProcessor(): Unit = { eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) } @@ -172,6 +198,16 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { .endTerminated() .endState() .endContainerStatus() + .addNewContainerStatus() + .withName("spark-executor-sidecar") + .withImage("k8s-spark-sidecar") + .withNewState() + .withNewTerminated() + .withMessage("Failed") + .withExitCode(1) + .endTerminated() + .endState() + .endContainerStatus() .withMessage("Executor failed.") .withReason("Executor failed because of a thrown error.") .endStatus() @@ -224,8 +260,8 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { val sparkPod = executorPodWithId(executorId.toString) val podWithAttachedContainer = new PodBuilder(sparkPod.pod) .editOrNewSpec() - .addToContainers(sparkPod.container) - .endSpec() + .addToContainers(sparkPod.container) + .endSpec() .build() podWithAttachedContainer } @@ -233,11 +269,11 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { private def executorPodWithId(executorId: String): SparkPod = { val pod = new PodBuilder() .withNewMetadata() - .withName(s"spark-executor-$executorId") - .addToLabels(SPARK_APP_ID_LABEL, appId) - .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) - .addToLabels(SPARK_EXECUTOR_ID_LABEL, executorId) - .endMetadata() + .withName(s"spark-executor-$executorId") + .addToLabels(SPARK_APP_ID_LABEL, appId) + .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .addToLabels(SPARK_EXECUTOR_ID_LABEL, executorId) + .endMetadata() .build() val container = new ContainerBuilder() .withName("spark-executor") From ca3fdb36e676805e97c07c3e900e9a5440933dd6 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 23 May 2018 16:58:21 -0700 Subject: [PATCH 15/47] Publish pod updates to the pod allocator and lifecycle manager separately. Uses ReactiveX as a utility to push the pod updates to the two modules at different intervals. --- dev/deps/spark-deps-hadoop-2.6 | 2 + dev/deps/spark-deps-hadoop-2.7 | 2 + dev/deps/spark-deps-hadoop-3.1 | 2 + pom.xml | 5 + resource-managers/kubernetes/core/pom.xml | 5 + .../cluster/k8s/ExecutorPodsAllocator.scala | 120 +++++++++ .../k8s/ExecutorPodsEventHandler.scala | 229 ------------------ .../cluster/k8s/ExecutorPodsEventQueue.scala | 97 ++++++++ .../ExecutorPodsLifecycleEventHandler.scala | 128 ++++++++++ .../k8s/ExecutorPodsPollingEventSource.scala | 10 +- .../k8s/ExecutorPodsWatchEventSource.scala | 4 +- .../k8s/KubernetesClusterManager.scala | 16 +- .../KubernetesClusterSchedulerBackend.scala | 16 +- .../k8s/ExecutorPodsEventHandlerSuite.scala | 27 +-- 14 files changed, 394 insertions(+), 269 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index e710e26348117..0161f7b832672 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -172,6 +172,8 @@ parquet-jackson-1.10.0.jar protobuf-java-2.5.0.jar py4j-0.10.7.jar pyrolite-4.13.jar +reactive-streams-1.0.2.jar +rxjava-2.1.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 97ad17a9ff7b1..d43566790ac15 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -173,6 +173,8 @@ parquet-jackson-1.10.0.jar protobuf-java-2.5.0.jar py4j-0.10.7.jar pyrolite-4.13.jar +reactive-streams-1.0.2.jar +rxjava-2.1.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index e21bfef8c4291..930791b00f8fc 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -192,6 +192,8 @@ protobuf-java-2.5.0.jar py4j-0.10.7.jar pyrolite-4.13.jar re2j-1.1.jar +reactive-streams-1.0.2.jar +rxjava-2.1.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/pom.xml b/pom.xml index 6fbf58dec9695..f3b084682f4a0 100644 --- a/pom.xml +++ b/pom.xml @@ -747,6 +747,11 @@ jline jline 2.12.1 + + + io.reactivex.rxjava2 + rxjava + 2.1.13 org.scalatest diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index a6dd47a6b7d95..def7be4060b97 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -83,6 +83,11 @@ 3.8.1 + + io.reactivex.rxjava2 + rxjava + + org.mockito mockito-core diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala new file mode 100644 index 0000000000000..cf97491959709 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler.cluster.k8s + +import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} + +import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} +import io.fabric8.kubernetes.client.KubernetesClient +import scala.collection.mutable + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.KubernetesConf +import org.apache.spark.internal.Logging + +private[spark] class ExecutorPodsAllocator( + conf: SparkConf, + executorBuilder: KubernetesExecutorBuilder, + kubernetesClient: KubernetesClient, + eventQueue: ExecutorPodsEventQueue) extends Logging { + + private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) + + private val totalExpectedExecutors = new AtomicInteger(0) + + private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) + + private val podAllocationDelay = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) + + private val kubernetesDriverPodName = conf + .get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse(throw new SparkException("Must specify the driver pod name")) + + private val driverPod = kubernetesClient.pods() + .withName(kubernetesDriverPodName) + .get() + + // Use sets of ids instead of counters to be able to handle duplicate events. + + // Executor IDs that have been requested from Kubernetes but are not running yet. + private val pendingExecutors = mutable.Set.empty[Long] + + // We could use CoarseGrainedSchedulerBackend#totalRegisteredExecutors here for tallying the + // executors that are running. But, here we choose instead to maintain all state within this + // class from the persecptive of the k8s API. Therefore whether or not this scheduler loop + // believes an executor is running is dictated by the K8s API rather than Spark's RPC events. + // We may need to consider where these perspectives may differ and which perspective should + // take precedence. + private val runningExecutors = mutable.Set.empty[Long] + + def start(applicationId: String): Unit = { + eventQueue.addSubscriber(podAllocationDelay) { updatedPods => + processUpdatedPodEvents(applicationId, updatedPods) + } + } + + def setTotalExpectedExecutors(total: Int): Unit = totalExpectedExecutors.set(total) + + private def processUpdatedPodEvents(applicationId: String, updatedPods: Seq[Pod]): Unit = { + updatedPods.foreach { updatedPod => + val execId = updatedPod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL).toLong + val phase = updatedPod.getStatus.getPhase.toLowerCase + phase match { + case "running" => + pendingExecutors -= execId + runningExecutors += execId + case "failed" | "succeeded" | "error" => + pendingExecutors -= execId + runningExecutors -= execId + } + } + + val currentRunningExecutors = runningExecutors.size + val currentTotalExpectedExecutors = totalExpectedExecutors.get + if (pendingExecutors.isEmpty && currentRunningExecutors < currentTotalExpectedExecutors) { + val numExecutorsToAllocate = math.min( + currentTotalExpectedExecutors - currentRunningExecutors, podAllocationSize) + logInfo(s"Going to request $numExecutorsToAllocate executors from Kubernetes.") + val newExecutorIds = mutable.Buffer.empty[Long] + val podsToAllocate = mutable.Buffer.empty[Pod] + for ( _ <- 0 until numExecutorsToAllocate) { + val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet() + val executorConf = KubernetesConf.createExecutorConf( + conf, + newExecutorId.toString, + applicationId, + driverPod) + val executorPod = executorBuilder.buildFromFeatures(executorConf) + val podWithAttachedContainer = new PodBuilder(executorPod.pod) + .editOrNewSpec() + .addToContainers(executorPod.container) + .endSpec() + .build() + kubernetesClient.pods().create(podWithAttachedContainer) + pendingExecutors += newExecutorId + } + } else if (currentRunningExecutors == currentTotalExpectedExecutors) { + logDebug("Current number of running executors is equal to the number of requested" + + " executors. Not scaling up further.") + } else if (pendingExecutors.nonEmpty) { + logInfo(s"Still waiting for ${pendingExecutors.size} executors to begin running before" + + s" requesting for more executors.") + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala deleted file mode 100644 index 3b5e9a3fe04ee..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandler.scala +++ /dev/null @@ -1,229 +0,0 @@ -/* - * 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.{Future, LinkedBlockingQueue, ScheduledExecutorService, TimeUnit} -import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} - -import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} -import io.fabric8.kubernetes.client.KubernetesClient -import scala.collection.JavaConverters._ -import scala.collection.mutable - -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesConf -import org.apache.spark.internal.Logging -import org.apache.spark.scheduler.ExecutorExited -import org.apache.spark.util.Utils - -private[spark] class ExecutorPodsEventHandler( - conf: SparkConf, - executorBuilder: KubernetesExecutorBuilder, - kubernetesClient: KubernetesClient, - eventProcessorExecutor: ScheduledExecutorService) extends Logging { - - import ExecutorPodsEventHandler._ - - private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) - - private val totalExpectedExecutors = new AtomicInteger(0) - - private val eventQueue = new LinkedBlockingQueue[Seq[Pod]]() - - private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) - - private val podAllocationDelay = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) - - private val kubernetesDriverPodName = conf - .get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse(throw new SparkException("Must specify the driver pod name")) - - private val driverPod = kubernetesClient.pods() - .withName(kubernetesDriverPodName) - .get() - - // Use sets of ids instead of counters to be able to handle duplicate events. - - // Executor IDs that have been requested from Kubernetes but are not running yet. - private val pendingExecutors = mutable.Set.empty[Long] - - // We could use CoarseGrainedSchedulerBackend#totalRegisteredExecutors here for tallying the - // executors that are running. But, here we choose instead to maintain all state within this - // class from the persecptive of the k8s API. Therefore whether or not this scheduler loop - // believes an executor is running is dictated by the K8s API rather than Spark's RPC events. - // We may need to consider where these perspectives may differ and which perspective should - // take precedence. - private val runningExecutors = mutable.Set.empty[Long] - - private var eventProcessorFuture: Future[_] = _ - - def start(applicationId: String, schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { - require(eventProcessorFuture == null, "Cannot start event processing twice.") - logInfo(s"Starting Kubernetes executor pods event handler for application with" + - s" id $applicationId.") - val eventProcessor = new Runnable { - override def run(): Unit = { - Utils.tryLogNonFatalError { - processEvents(applicationId, schedulerBackend) - } - } - } - eventProcessorFuture = eventProcessorExecutor.scheduleWithFixedDelay( - eventProcessor, 0L, podAllocationDelay, TimeUnit.MILLISECONDS) - } - - def stop(): Unit = { - if (eventProcessorFuture != null) { - eventProcessorFuture.cancel(true) - eventProcessorFuture = null - } - } - - private def processEvents( - applicationId: String, schedulerBackend: KubernetesClusterSchedulerBackend) { - val currentEvents = new java.util.ArrayList[Seq[Pod]](eventQueue.size()) - eventQueue.drainTo(currentEvents) - currentEvents.asScala.flatten.foreach { updatedPod => - val execId = updatedPod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL).toLong - val podPhase = updatedPod.getStatus.getPhase.toLowerCase - if (isDeleted(updatedPod)) { - removeExecutorFromSpark(schedulerBackend, updatedPod, execId) - } else { - updatedPod.getStatus.getPhase.toLowerCase match { - case "running" => - // If clause is for resililence to out of order operations - executor must be - // pending and first reach running. Without this check you may e.g. process a - // deletion event followed by some arbitrary modification event - we want the - // deletion event to "stick". - if (pendingExecutors.contains(execId)) { - pendingExecutors.remove(execId) - runningExecutors.add(execId) - } - // TODO (SPARK-24135) - handle more classes of errors - case "error" | "failed" | "succeeded" => - // If deletion failed on a previous try, we can try again if resync informs us the pod - // is still around. - // Delete as best attempt - duplicate deletes will throw an exception but the end state - // of getting rid of the pod is what matters. - if (!isDeleted(updatedPod)) { - Utils.tryLogNonFatalError { - kubernetesClient - .pods() - .withName(updatedPod.getMetadata.getName) - .delete() - } - } - removeExecutorFromSpark(schedulerBackend, updatedPod, execId) - } - } - } - - val currentRunningExecutors = runningExecutors.size - val currentTotalExpectedExecutors = totalExpectedExecutors.get - if (pendingExecutors.isEmpty && currentRunningExecutors < currentTotalExpectedExecutors) { - val numExecutorsToAllocate = math.min( - currentTotalExpectedExecutors - currentRunningExecutors, podAllocationSize) - logInfo(s"Going to request $numExecutorsToAllocate executors from Kubernetes.") - val newExecutorIds = mutable.Buffer.empty[Long] - val podsToAllocate = mutable.Buffer.empty[Pod] - for ( _ <- 0 until numExecutorsToAllocate) { - val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet() - val executorConf = KubernetesConf.createExecutorConf( - conf, - newExecutorId.toString, - applicationId, - driverPod) - val executorPod = executorBuilder.buildFromFeatures(executorConf) - val podWithAttachedContainer = new PodBuilder(executorPod.pod) - .editOrNewSpec() - .addToContainers(executorPod.container) - .endSpec() - .build() - kubernetesClient.pods().create(podWithAttachedContainer) - pendingExecutors += newExecutorId - } - } else if (currentRunningExecutors == currentTotalExpectedExecutors) { - logDebug("Current number of running executors is equal to the number of requested" + - " executors. Not scaling up further.") - } else if (pendingExecutors.nonEmpty) { - logInfo(s"Still waiting for ${pendingExecutors.size} executors to begin running before" + - s" requesting for more executors.") - } - } - - def sendUpdatedPodMetadata(updatedPod: Pod): Unit = { - eventQueue.add(Seq(updatedPod)) - } - - def sendUpdatedPodMetadata(updatedPods: Iterable[Pod]): Unit = { - eventQueue.add(updatedPods.toSeq) - } - - def setTotalExpectedExecutors(newTotal: Int): Unit = totalExpectedExecutors.set(newTotal) - - private def removeExecutorFromSpark( - schedulerBackend: KubernetesClusterSchedulerBackend, - updatedPod: Pod, - execId: Long): Unit = { - // Avoid removing twice from Spark's perspective. - if (pendingExecutors.contains(execId) || runningExecutors.contains(execId)) { - pendingExecutors.remove(execId) - runningExecutors.remove(execId) - val exitReason = findExitReason(updatedPod, execId) - schedulerBackend.doRemoveExecutor(execId.toString, exitReason) - } - } - - private def findExitReason(pod: Pod, execId: Long): ExecutorExited = { - val exitCode = findExitCode(pod) - val (exitCausedByApp, exitMessage) = if (isDeleted(pod)) { - (false, s"The executor with id $execId was deleted by a user or the framework.") - } else { - val msg = exitReasonMessage(pod, execId, exitCode) - (true, msg) - } - ExecutorExited(exitCode, exitCausedByApp, exitMessage) - } - - private def exitReasonMessage(pod: Pod, execId: Long, exitCode: Int) = { - s""" - |The executor with id $execId exited with exit code $exitCode. - |The API gave the following brief reason: ${pod.getStatus.getReason} - |The API gave the following message: ${pod.getStatus.getMessage} - |The API gave the following container statuses: - | - |${pod.getStatus.getContainerStatuses.asScala.map(_.toString).mkString("\n===\n")} - """.stripMargin - } - - private def isDeleted(pod: Pod): Boolean = pod.getMetadata.getDeletionTimestamp != null - - private def findExitCode(pod: Pod): Int = { - pod.getStatus.getContainerStatuses.asScala.find { containerStatus => - containerStatus.getState.getTerminated != null - }.map { terminatedContainer => - terminatedContainer.getState.getTerminated.getExitCode.toInt - }.getOrElse(UNKNOWN_EXIT_CODE) - } -} - -private object ExecutorPodsEventHandler { - val UNKNOWN_EXIT_CODE = -1 -} - diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala new file mode 100644 index 0000000000000..aa3f4a35209c2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala @@ -0,0 +1,97 @@ +/* + * 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.{Callable, LinkedBlockingQueue, ScheduledExecutorService, TimeUnit} +import java.util.concurrent.atomic.AtomicBoolean + +import io.fabric8.kubernetes.api.model.Pod +import io.reactivex.Flowable +import io.reactivex.disposables.Disposable +import io.reactivex.functions.{BooleanSupplier, Consumer} +import io.reactivex.schedulers.Schedulers +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.util.Utils + +private[spark] class ExecutorPodsEventQueue(eventsProcessorExecutor: ScheduledExecutorService) { + + private val eventQueue = new LinkedBlockingQueue[Pod]() + private val terminationSignal = new AtomicBoolean(false) + private val eventsObservable = + // Source is from the blocking queue + Flowable.fromCallable(toCallable(eventQueue.take())) + // Keep polling for items until we're told to stop. When the event queue is empty we'll + // be able to stall, preventing overload of the downstream observables. + .repeatUntil(toReactivexBooleanSupplier(() => terminationSignal.get())) + // When the call to future.get() returns, halt the event stream + // Forces every event to be shared amongst all observers. Will + .publish() + private val observedDisposables = mutable.Buffer.empty[Disposable] + + def addSubscriber(processBatchIntervalMillis: Long)(onNextBatch: Seq[Pod] => Unit): Unit = { + observedDisposables += eventsObservable + // Group events in the time window given by the caller. These buffers are then sent + // to the caller's lambda at the given interval, with the pod updates that occurred + // in that given interval. + .buffer( + processBatchIntervalMillis, + TimeUnit.MILLISECONDS, + // For testing - specifically use the given scheduled executor service to trigger + // buffer boundaries. Allows us to inject a deterministic scheduler here. + Schedulers.from(eventsProcessorExecutor)) + .subscribeOn(Schedulers.from(eventsProcessorExecutor)) + .subscribe(toReactivexConsumer { (pods: java.util.List[Pod]) => + Utils.tryLogNonFatalError { + onNextBatch(pods.asScala) + } + }) + } + + def startProcessingEvents(): Unit = eventsObservable.connect() + + def stopProcessingEvents(): Unit = { + terminationSignal.set(true) + observedDisposables.foreach { disposable => + Utils.tryLogNonFatalError { + disposable.dispose() + } + } + eventsProcessorExecutor.shutdownNow() + } + + def pushPodUpdate(updatedPod: Pod): Unit = eventQueue.add(updatedPod) + + private def toCallable[T](callable: => T): Callable[T] = { + new Callable[T] { + override def call(): T = callable + } + } + + private def toReactivexConsumer[T](consumer: T => Unit): Consumer[T] = { + new Consumer[T] { + override def accept(item: T): Unit = consumer(item) + } + } + + private def toReactivexBooleanSupplier(supplier: () => Boolean): BooleanSupplier = { + new BooleanSupplier { + override def getAsBoolean = supplier() + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala new file mode 100644 index 0000000000000..1fe777ac6e51e --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala @@ -0,0 +1,128 @@ +/* + * 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.{Future, TimeUnit} + +import com.google.common.cache.CacheBuilder +import io.fabric8.kubernetes.api.model.Pod +import io.fabric8.kubernetes.client.KubernetesClient +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkConf +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.ExecutorExited +import org.apache.spark.util.Utils + +private[spark] class ExecutorPodsLifecycleEventHandler( + conf: SparkConf, + executorBuilder: KubernetesExecutorBuilder, + kubernetesClient: KubernetesClient, + podsEventQueue: ExecutorPodsEventQueue) extends Logging { + + import ExecutorPodsLifecycleEventHandler._ + + // Use a best-effort to track which executors have been removed already. It's not generally + // job-breaking if we remove executors more than once but it's ideal if we make an attempt + // to avoid doing so. Expire cache entries so that this data structure doesn't grow beyond + // bounds. + private val removedExecutors = CacheBuilder.newBuilder() + .expireAfterWrite(3, TimeUnit.MINUTES) + .build[Long, java.lang.Long] + + def start(schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { + podsEventQueue.addSubscriber(1000L) { updatedPods => + updatedPods.foreach { updatedPod => + processUpdatedPod(schedulerBackend, updatedPod) + } + } + } + + private def processUpdatedPod( + schedulerBackend: KubernetesClusterSchedulerBackend, updatedPod: Pod) = { + val execId = updatedPod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL).toLong + if (isDeleted(updatedPod)) { + removeExecutorFromSpark(schedulerBackend, updatedPod, execId) + } else { + updatedPod.getStatus.getPhase.toLowerCase match { + // TODO (SPARK-24135) - handle more classes of errors + case "error" | "failed" | "succeeded" => + // If deletion failed on a previous try, we can try again if resync informs us the pod + // is still around. + // Delete as best attempt - duplicate deletes will throw an exception but the end state + // of getting rid of the pod is what matters. + Utils.tryLogNonFatalError { + kubernetesClient + .pods() + .withName(updatedPod.getMetadata.getName) + .delete() + } + removeExecutorFromSpark(schedulerBackend, updatedPod, execId) + } + } + } + + private def removeExecutorFromSpark( + schedulerBackend: KubernetesClusterSchedulerBackend, + updatedPod: Pod, + execId: Long): Unit = { + if (removedExecutors.getIfPresent(execId) == null) { + removedExecutors.put(execId, execId) + val exitReason = findExitReason(updatedPod, execId) + schedulerBackend.doRemoveExecutor(execId.toString, exitReason) + } + } + + private def findExitReason(pod: Pod, execId: Long): ExecutorExited = { + val exitCode = findExitCode(pod) + val (exitCausedByApp, exitMessage) = if (isDeleted(pod)) { + (false, s"The executor with id $execId was deleted by a user or the framework.") + } else { + val msg = exitReasonMessage(pod, execId, exitCode) + (true, msg) + } + ExecutorExited(exitCode, exitCausedByApp, exitMessage) + } + + private def exitReasonMessage(pod: Pod, execId: Long, exitCode: Int) = { + s""" + |The executor with id $execId exited with exit code $exitCode. + |The API gave the following brief reason: ${pod.getStatus.getReason} + |The API gave the following message: ${pod.getStatus.getMessage} + |The API gave the following container statuses: + | + |${pod.getStatus.getContainerStatuses.asScala.map(_.toString).mkString("\n===\n")} + """.stripMargin + } + + private def isDeleted(pod: Pod): Boolean = pod.getMetadata.getDeletionTimestamp != null + + private def findExitCode(pod: Pod): Int = { + pod.getStatus.getContainerStatuses.asScala.find { containerStatus => + containerStatus.getState.getTerminated != null + }.map { terminatedContainer => + terminatedContainer.getState.getTerminated.getExitCode.toInt + }.getOrElse(UNKNOWN_EXIT_CODE) + } +} + +private object ExecutorPodsLifecycleEventHandler { + val UNKNOWN_EXIT_CODE = -1 +} + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala index 05462fc89ee7b..7c9b2b04a0f35 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala @@ -25,10 +25,9 @@ import org.apache.spark.deploy.k8s.Constants._ private[spark] class ExecutorPodsPollingEventSource( kubernetesClient: KubernetesClient, - eventHandler: ExecutorPodsEventHandler, + eventQueue: ExecutorPodsEventQueue, pollingExecutor: ScheduledExecutorService) { - - private var pollingFuture: Future[_] = null + private var pollingFuture: Future[_] = _ def start(applicationId: String): Unit = { require(pollingFuture == null, "Cannot start polling more than once.") @@ -45,13 +44,14 @@ private[spark] class ExecutorPodsPollingEventSource( private class PollRunnable(applicationId: String) extends Runnable { override def run(): Unit = { - eventHandler.sendUpdatedPodMetadata(kubernetesClient + kubernetesClient .pods() .withLabel(SPARK_APP_ID_LABEL, applicationId) .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .list() .getItems - .asScala) + .asScala + .foreach(eventQueue.pushPodUpdate) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala index 48e4c61d3a600..288ea326c4af7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala @@ -27,7 +27,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.util.Utils private[spark] class ExecutorPodsWatchEventSource( - eventHandler: ExecutorPodsEventHandler, + podsEventQueue: ExecutorPodsEventQueue, kubernetesClient: KubernetesClient) extends Logging { private var watchConnection: Closeable = null @@ -51,7 +51,7 @@ private[spark] class ExecutorPodsWatchEventSource( private class ExecutorPodsWatcher extends Watcher[Pod] { override def eventReceived(action: Action, pod: Pod): Unit = { - eventHandler.sendUpdatedPodMetadata(pod) + podsEventQueue.pushPodUpdate(pod) } override def onClose(e: KubernetesClientException): Unit = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 710001ea02e50..5165775dfb685 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -61,27 +61,33 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val eventsProcessorExecutor = ThreadUtils .newDaemonSingleThreadScheduledExecutor("kubernetes-executor-pods-event-handler") - val executorPodsEventHandler = new ExecutorPodsEventHandler( + val eventQueue = new ExecutorPodsEventQueue(eventsProcessorExecutor) + val executorPodsLifecycleEventHandler = new ExecutorPodsLifecycleEventHandler( sc.conf, new KubernetesExecutorBuilder(), kubernetesClient, - eventsProcessorExecutor) + eventQueue) + + val executorPodsAllocator = new ExecutorPodsAllocator( + sc.conf, new KubernetesExecutorBuilder(), kubernetesClient, eventQueue) val podsWatchEventSource = new ExecutorPodsWatchEventSource( - executorPodsEventHandler, + eventQueue, kubernetesClient) val eventsPollingExecutor = ThreadUtils.newDaemonSingleThreadScheduledExecutor( "kubernetes-executor-pod-polling-sync") val podsPollingEventSource = new ExecutorPodsPollingEventSource( - kubernetesClient, executorPodsEventHandler, eventsPollingExecutor) + kubernetesClient, eventQueue, eventsPollingExecutor) new KubernetesClusterSchedulerBackend( scheduler.asInstanceOf[TaskSchedulerImpl], sc.env.rpcEnv, kubernetesClient, requestExecutorsService, - executorPodsEventHandler, + eventQueue, + executorPodsAllocator, + executorPodsLifecycleEventHandler, podsWatchEventSource, podsPollingEventSource) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 87898418c2fdc..06085053ee407 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -32,7 +32,9 @@ private[spark] class KubernetesClusterSchedulerBackend( rpcEnv: RpcEnv, kubernetesClient: KubernetesClient, requestExecutorsService: ExecutorService, - eventHandler: ExecutorPodsEventHandler, + eventQueue: ExecutorPodsEventQueue, + podAllocator: ExecutorPodsAllocator, + lifecycleEventHandler: ExecutorPodsLifecycleEventHandler, watchEvents: ExecutorPodsWatchEventSource, pollEvents: ExecutorPodsPollingEventSource) extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { @@ -57,19 +59,19 @@ private[spark] class KubernetesClusterSchedulerBackend( override def start(): Unit = { super.start() if (!Utils.isDynamicAllocationEnabled(conf)) { - eventHandler.setTotalExpectedExecutors(initialExecutors) + podAllocator.setTotalExpectedExecutors(initialExecutors) } - eventHandler.start(applicationId(), this) watchEvents.start(applicationId()) pollEvents.start(applicationId()) + lifecycleEventHandler.start(this) + podAllocator.start(applicationId()) + eventQueue.startProcessingEvents() } override def stop(): Unit = { super.stop() - Utils.tryLogNonFatalError { - eventHandler.stop() - } + eventQueue.stopProcessingEvents() Utils.tryLogNonFatalError { watchEvents.stop() @@ -92,7 +94,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { - eventHandler.setTotalExpectedExecutors(requestedTotal) + lifecycleEventHandler.setTotalExpectedExecutors(requestedTotal) true } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala index 40eb5993ead09..a7c3095308c66 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala @@ -16,30 +16,14 @@ */ package org.apache.spark.scheduler.cluster.k8s -import java.util.concurrent.TimeUnit - -import io.fabric8.kubernetes.api.model.{ContainerBuilder, DoneablePod, Pod, PodBuilder, PodList} -import io.fabric8.kubernetes.client.KubernetesClient -import io.fabric8.kubernetes.client.dsl.{MixedOperation, PodResource} -import org.jmock.lib.concurrent.DeterministicScheduler -import org.mockito.{ArgumentMatcher, Matchers, MockitoAnnotations} -import org.mockito.Matchers.any -import org.mockito.Mockito.{mock, never, times, verify, when} -import org.mockito.MockitoAnnotations.Mock -import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter -import scala.collection.JavaConverters._ -import scala.collection.mutable -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} -import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.scheduler.ExecutorExited +import org.apache.spark.SparkFunSuite class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { + // TODO + /* private type Pods = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] private val driverPodName = "driver" @@ -80,7 +64,7 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { @Mock private var schedulerBackend: KubernetesClusterSchedulerBackend = _ - private var eventHandlerUnderTest: ExecutorPodsEventHandler = _ + private var eventHandlerUnderTest: ExecutorPodsLifecycleEventHandler = _ before { MockitoAnnotations.initMocks(this) @@ -91,7 +75,7 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { when(driverPodOperations.get).thenReturn(driverPod) when(executorBuilder.buildFromFeatures(kubernetesConfWithCorrectFields())) .thenAnswer(executorPodAnswer()) - eventHandlerUnderTest = new ExecutorPodsEventHandler( + eventHandlerUnderTest = new ExecutorPodsLifecycleEventHandler( conf, executorBuilder, kubernetesClient, eventProcessorExecutor) eventHandlerUnderTest.start(appId, schedulerBackend) } @@ -281,4 +265,5 @@ class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { .build() SparkPod(pod, container) } + */ } From 79ebaf345a56c7d4a3353629a14f71c5eb720c65 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 23 May 2018 17:12:50 -0700 Subject: [PATCH 16/47] Ensure we make a request round immediately --- .../scheduler/cluster/k8s/ExecutorPodsAllocator.scala | 9 +++++++++ .../k8s/KubernetesClusterSchedulerBackend.scala | 11 +++++++---- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index cf97491959709..b6e6bfe24dc34 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -69,6 +69,15 @@ private[spark] class ExecutorPodsAllocator( } } + def start(applicationId: String, initialExecutors: Int): Unit = { + // Bootstrap immediately - the subsequent requests will only occur asynchronously after the + // given delay. + setTotalExpectedExecutors(initialExecutors) + processUpdatedPodEvents(applicationId, Seq.empty[Pod]) + + start(applicationId) + } + def setTotalExpectedExecutors(total: Int): Unit = totalExpectedExecutors.set(total) private def processUpdatedPodEvents(applicationId: String, updatedPods: Seq[Pod]): Unit = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 06085053ee407..1c29c2078cf3e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -58,13 +58,16 @@ private[spark] class KubernetesClusterSchedulerBackend( override def start(): Unit = { super.start() - if (!Utils.isDynamicAllocationEnabled(conf)) { - podAllocator.setTotalExpectedExecutors(initialExecutors) - } watchEvents.start(applicationId()) pollEvents.start(applicationId()) lifecycleEventHandler.start(this) - podAllocator.start(applicationId()) + + if (!Utils.isDynamicAllocationEnabled(conf)) { + podAllocator.start(applicationId(), initialExecutors) + } else { + podAllocator.start(applicationId()) + } + eventQueue.startProcessingEvents() } From fadbe9f870dbe33a5bb6a485d891bc87cbd05dea Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 23 May 2018 17:15:18 -0700 Subject: [PATCH 17/47] Remove a comment --- .../spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala index aa3f4a35209c2..263b3b142735a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala @@ -39,7 +39,6 @@ private[spark] class ExecutorPodsEventQueue(eventsProcessorExecutor: ScheduledEx // Keep polling for items until we're told to stop. When the event queue is empty we'll // be able to stall, preventing overload of the downstream observables. .repeatUntil(toReactivexBooleanSupplier(() => terminationSignal.get())) - // When the call to future.get() returns, halt the event stream // Forces every event to be shared amongst all observers. Will .publish() private val observedDisposables = mutable.Buffer.empty[Disposable] From 4f583939f9e3d6d1df7a0d44ec0c5acf6ae82ef1 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 23 May 2018 17:21:18 -0700 Subject: [PATCH 18/47] Fix compilation --- .../cluster/k8s/ExecutorPodsLifecycleEventHandler.scala | 2 +- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala index 1fe777ac6e51e..6443011876aa5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala @@ -44,7 +44,7 @@ private[spark] class ExecutorPodsLifecycleEventHandler( // bounds. private val removedExecutors = CacheBuilder.newBuilder() .expireAfterWrite(3, TimeUnit.MINUTES) - .build[Long, java.lang.Long] + .build[java.lang.Long, java.lang.Long]() def start(schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { podsEventQueue.addSubscriber(1000L) { updatedPods => diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 1c29c2078cf3e..f2740e5df01bd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -97,7 +97,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { - lifecycleEventHandler.setTotalExpectedExecutors(requestedTotal) + podAllocator.setTotalExpectedExecutors(requestedTotal) true } From 2a2374c915aafa1b5a53c8e02581cea0c2c176df Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 23 May 2018 17:23:04 -0700 Subject: [PATCH 19/47] Don't use tabs --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f3b084682f4a0..1a9f2ff0c1589 100644 --- a/pom.xml +++ b/pom.xml @@ -748,7 +748,7 @@ jline 2.12.1 - + io.reactivex.rxjava2 rxjava 2.1.13 From 5850439652fad6bb2b03daf4e35497304c8defdd Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 24 May 2018 12:31:05 -0700 Subject: [PATCH 20/47] Use PublishSubject instead of a LinkedBlockingQueue at all --- .../cluster/k8s/ExecutorPodsAllocator.scala | 9 -- .../cluster/k8s/ExecutorPodsEventQueue.scala | 46 ++--------- .../KubernetesClusterSchedulerBackend.scala | 14 ++-- .../k8s/ExecutorPodsEventQueueSuite.scala | 82 +++++++++++++++++++ 4 files changed, 96 insertions(+), 55 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index b6e6bfe24dc34..cf97491959709 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -69,15 +69,6 @@ private[spark] class ExecutorPodsAllocator( } } - def start(applicationId: String, initialExecutors: Int): Unit = { - // Bootstrap immediately - the subsequent requests will only occur asynchronously after the - // given delay. - setTotalExpectedExecutors(initialExecutors) - processUpdatedPodEvents(applicationId, Seq.empty[Pod]) - - start(applicationId) - } - def setTotalExpectedExecutors(total: Int): Unit = totalExpectedExecutors.set(total) private def processUpdatedPodEvents(applicationId: String, updatedPods: Seq[Pod]): Unit = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala index 263b3b142735a..00a5416b4e8f6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala @@ -16,14 +16,14 @@ */ package org.apache.spark.scheduler.cluster.k8s -import java.util.concurrent.{Callable, LinkedBlockingQueue, ScheduledExecutorService, TimeUnit} -import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.{ScheduledExecutorService, TimeUnit} +import com.google.common.collect.Lists import io.fabric8.kubernetes.api.model.Pod -import io.reactivex.Flowable import io.reactivex.disposables.Disposable -import io.reactivex.functions.{BooleanSupplier, Consumer} +import io.reactivex.functions.Consumer import io.reactivex.schedulers.Schedulers +import io.reactivex.subjects.PublishSubject import scala.collection.JavaConverters._ import scala.collection.mutable @@ -31,16 +31,7 @@ import org.apache.spark.util.Utils private[spark] class ExecutorPodsEventQueue(eventsProcessorExecutor: ScheduledExecutorService) { - private val eventQueue = new LinkedBlockingQueue[Pod]() - private val terminationSignal = new AtomicBoolean(false) - private val eventsObservable = - // Source is from the blocking queue - Flowable.fromCallable(toCallable(eventQueue.take())) - // Keep polling for items until we're told to stop. When the event queue is empty we'll - // be able to stall, preventing overload of the downstream observables. - .repeatUntil(toReactivexBooleanSupplier(() => terminationSignal.get())) - // Forces every event to be shared amongst all observers. Will - .publish() + private val eventsObservable = PublishSubject.create[Pod]() private val observedDisposables = mutable.Buffer.empty[Disposable] def addSubscriber(processBatchIntervalMillis: Long)(onNextBatch: Seq[Pod] => Unit): Unit = { @@ -54,7 +45,7 @@ private[spark] class ExecutorPodsEventQueue(eventsProcessorExecutor: ScheduledEx // For testing - specifically use the given scheduled executor service to trigger // buffer boundaries. Allows us to inject a deterministic scheduler here. Schedulers.from(eventsProcessorExecutor)) - .subscribeOn(Schedulers.from(eventsProcessorExecutor)) + .startWith(Lists.newArrayList[Pod]()) .subscribe(toReactivexConsumer { (pods: java.util.List[Pod]) => Utils.tryLogNonFatalError { onNextBatch(pods.asScala) @@ -62,35 +53,16 @@ private[spark] class ExecutorPodsEventQueue(eventsProcessorExecutor: ScheduledEx }) } - def startProcessingEvents(): Unit = eventsObservable.connect() - def stopProcessingEvents(): Unit = { - terminationSignal.set(true) - observedDisposables.foreach { disposable => - Utils.tryLogNonFatalError { - disposable.dispose() - } - } - eventsProcessorExecutor.shutdownNow() + observedDisposables.foreach(_.dispose()) + eventsObservable.onComplete() } - def pushPodUpdate(updatedPod: Pod): Unit = eventQueue.add(updatedPod) - - private def toCallable[T](callable: => T): Callable[T] = { - new Callable[T] { - override def call(): T = callable - } - } + def pushPodUpdate(updatedPod: Pod): Unit = eventsObservable.onNext(updatedPod) private def toReactivexConsumer[T](consumer: T => Unit): Consumer[T] = { new Consumer[T] { override def accept(item: T): Unit = consumer(item) } } - - private def toReactivexBooleanSupplier(supplier: () => Boolean): BooleanSupplier = { - new BooleanSupplier { - override def getAsBoolean = supplier() - } - } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index f2740e5df01bd..a836c57c56627 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -58,17 +58,13 @@ private[spark] class KubernetesClusterSchedulerBackend( override def start(): Unit = { super.start() - watchEvents.start(applicationId()) - pollEvents.start(applicationId()) - lifecycleEventHandler.start(this) - if (!Utils.isDynamicAllocationEnabled(conf)) { - podAllocator.start(applicationId(), initialExecutors) - } else { - podAllocator.start(applicationId()) + podAllocator.setTotalExpectedExecutors(initialExecutors) } - - eventQueue.startProcessingEvents() + lifecycleEventHandler.start(this) + podAllocator.start(applicationId()) + watchEvents.start(applicationId()) + pollEvents.start(applicationId()) } override def stop(): Unit = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala new file mode 100644 index 0000000000000..2efc20f447afa --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala @@ -0,0 +1,82 @@ +/* + * 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.TimeUnit +import java.util.concurrent.atomic.AtomicReference + +import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} +import org.jmock.lib.concurrent.DeterministicScheduler +import org.scalatest.BeforeAndAfter +import scala.collection.mutable + +import org.apache.spark.SparkFunSuite + +class ExecutorPodsEventQueueSuite extends SparkFunSuite with BeforeAndAfter { + + private var eventProcessor: DeterministicScheduler = _ + private var eventQueueUnderTest: ExecutorPodsEventQueue = _ + + before { + eventProcessor = new DeterministicScheduler() + eventQueueUnderTest = new ExecutorPodsEventQueue(eventProcessor) + } + + test("Subscribers get notified of events periodically.") { + val receivedEvents1 = mutable.Buffer.empty[Pod] + val receivedEvents2 = mutable.Buffer.empty[Pod] + eventQueueUnderTest.addSubscriber(1000) { receivedEvents1.appendAll(_) } + eventQueueUnderTest.addSubscriber(2000) { receivedEvents2.appendAll(_) } + + pushPodWithIndex(1) + assert(receivedEvents1.isEmpty) + assert(receivedEvents2.isEmpty) + eventProcessor.tick(1000, TimeUnit.MILLISECONDS) + assertIndicesMatch(receivedEvents1, 1) + assert(receivedEvents2.isEmpty) + eventProcessor.tick(1000, TimeUnit.MILLISECONDS) + assertIndicesMatch(receivedEvents2, 1) + pushPodWithIndex(2) + pushPodWithIndex(3) + eventProcessor.tick(1000, TimeUnit.MILLISECONDS) + assertIndicesMatch(receivedEvents1, 1, 2, 3) + assertIndicesMatch(receivedEvents2, 1) + eventProcessor.tick(2000, TimeUnit.MILLISECONDS) + assertIndicesMatch(receivedEvents1, 1, 2, 3) + assertIndicesMatch(receivedEvents2, 1, 2, 3) + } + + test("Even without sending events, initially receive an empty buffer.") { + val receivedInitialBuffer = new AtomicReference[Seq[Pod]](null) + eventQueueUnderTest.addSubscriber(1000) { receivedInitialBuffer.set } + assert(receivedInitialBuffer.get != null) + } + + private def assertIndicesMatch(buffer: mutable.Buffer[Pod], indices: Int*): Unit = { + assert(buffer === indices.map(podWithIndex)) + } + + private def pushPodWithIndex(index: Int): Unit = + eventQueueUnderTest.pushPodUpdate(podWithIndex(index)) + + private def podWithIndex(index: Int): Pod = + new PodBuilder() + .editOrNewMetadata() + .withName(s"pod-$index") + .endMetadata() + .build() +} From d4cf40f715b7d6ad8b9d9e3cf9757b2d439f25ea Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 24 May 2018 14:59:07 -0700 Subject: [PATCH 21/47] Add tests. Adjust observable concurrency. --- .../cluster/k8s/ExecutorPodsEventQueue.scala | 47 +-- .../k8s/ExecutorPodsEventQueueImpl.scala | 77 +++++ .../ExecutorPodsLifecycleEventHandler.scala | 29 +- .../k8s/KubernetesClusterManager.scala | 14 +- .../KubernetesClusterSchedulerBackend.scala | 2 +- .../DeterministicExecutorPodsEventQueue.scala | 41 +++ .../k8s/ExecutorLifecycleTestUtils.scala | 91 ++++++ .../k8s/ExecutorPodsAllocatorSuite.scala | 163 +++++++++++ .../k8s/ExecutorPodsEventHandlerSuite.scala | 269 ------------------ .../k8s/ExecutorPodsEventQueueSuite.scala | 31 +- ...ecutorPodsLifecycleEventHandlerSuite.scala | 113 ++++++++ 11 files changed, 532 insertions(+), 345 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala index 00a5416b4e8f6..fde187cdc30d8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala @@ -16,53 +16,14 @@ */ package org.apache.spark.scheduler.cluster.k8s -import java.util.concurrent.{ScheduledExecutorService, TimeUnit} - -import com.google.common.collect.Lists import io.fabric8.kubernetes.api.model.Pod -import io.reactivex.disposables.Disposable -import io.reactivex.functions.Consumer -import io.reactivex.schedulers.Schedulers -import io.reactivex.subjects.PublishSubject -import scala.collection.JavaConverters._ -import scala.collection.mutable - -import org.apache.spark.util.Utils - -private[spark] class ExecutorPodsEventQueue(eventsProcessorExecutor: ScheduledExecutorService) { - private val eventsObservable = PublishSubject.create[Pod]() - private val observedDisposables = mutable.Buffer.empty[Disposable] +private[spark] trait ExecutorPodsEventQueue { - def addSubscriber(processBatchIntervalMillis: Long)(onNextBatch: Seq[Pod] => Unit): Unit = { - observedDisposables += eventsObservable - // Group events in the time window given by the caller. These buffers are then sent - // to the caller's lambda at the given interval, with the pod updates that occurred - // in that given interval. - .buffer( - processBatchIntervalMillis, - TimeUnit.MILLISECONDS, - // For testing - specifically use the given scheduled executor service to trigger - // buffer boundaries. Allows us to inject a deterministic scheduler here. - Schedulers.from(eventsProcessorExecutor)) - .startWith(Lists.newArrayList[Pod]()) - .subscribe(toReactivexConsumer { (pods: java.util.List[Pod]) => - Utils.tryLogNonFatalError { - onNextBatch(pods.asScala) - } - }) - } + def addSubscriber(processBatchIntervalMillis: Long)(onNextBatch: Seq[Pod] => Unit): Unit - def stopProcessingEvents(): Unit = { - observedDisposables.foreach(_.dispose()) - eventsObservable.onComplete() - } + def stopProcessingEvents(): Unit - def pushPodUpdate(updatedPod: Pod): Unit = eventsObservable.onNext(updatedPod) + def pushPodUpdate(updatedPod: Pod): Unit - private def toReactivexConsumer[T](consumer: T => Unit): Consumer[T] = { - new Consumer[T] { - override def accept(item: T): Unit = consumer(item) - } - } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala new file mode 100644 index 0000000000000..124f724890bd4 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala @@ -0,0 +1,77 @@ +/* + * 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.{Executor, ScheduledExecutorService, TimeUnit} + +import com.google.common.collect.Lists +import io.fabric8.kubernetes.api.model.Pod +import io.reactivex.disposables.Disposable +import io.reactivex.functions.Consumer +import io.reactivex.schedulers.Schedulers +import io.reactivex.subjects.PublishSubject +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.util.Utils + +private[spark] class ExecutorPodsEventQueueImpl( + bufferEventsExecutor: ScheduledExecutorService, + executeSubscriptionsExecutor: Executor) + extends ExecutorPodsEventQueue { + + private val eventsObservable = PublishSubject.create[Pod]() + private val observedDisposables = mutable.Buffer.empty[Disposable] + + def addSubscriber(processBatchIntervalMillis: Long)(onNextBatch: Seq[Pod] => Unit): Unit = { + observedDisposables += eventsObservable + // Group events in the time window given by the caller. These buffers are then sent + // to the caller's lambda at the given interval, with the pod updates that occurred + // in that given interval. + .buffer( + processBatchIntervalMillis, + TimeUnit.MILLISECONDS, + // For testing - specifically use the given scheduled executor service to trigger + // buffer boundaries. Allows us to inject a deterministic scheduler here. + Schedulers.from(bufferEventsExecutor)) + // Trigger an event cycle immediately. Not strictly required to be fully correct, but + // in particular the pod allocator should try to request executors immediately instead + // of waiting for one pod allocation delay. + .startWith(Lists.newArrayList[Pod]()) + // Force all triggered events - both the initial event above and the buffered ones in + // the following time windows - to execute asynchronously to this call's thread. + .observeOn(Schedulers.from(executeSubscriptionsExecutor)) + .subscribe(toReactivexConsumer { (pods: java.util.List[Pod]) => + Utils.tryLogNonFatalError { + onNextBatch(pods.asScala) + } + }) + } + + def stopProcessingEvents(): Unit = { + observedDisposables.foreach(_.dispose()) + eventsObservable.onComplete() + } + + def pushPodUpdate(updatedPod: Pod): Unit = eventsObservable.onNext(updatedPod) + + private def toReactivexConsumer[T](consumer: T => Unit): Consumer[T] = { + new Consumer[T] { + override def accept(item: T): Unit = consumer(item) + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala index 6443011876aa5..beb758466e789 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala @@ -16,36 +16,33 @@ */ package org.apache.spark.scheduler.cluster.k8s -import java.util.concurrent.{Future, TimeUnit} +import java.util.concurrent.TimeUnit -import com.google.common.cache.CacheBuilder +import com.google.common.cache.{Cache, CacheBuilder} import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.JavaConverters._ -import org.apache.spark.SparkConf -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.ExecutorExited import org.apache.spark.util.Utils private[spark] class ExecutorPodsLifecycleEventHandler( - conf: SparkConf, executorBuilder: KubernetesExecutorBuilder, kubernetesClient: KubernetesClient, - podsEventQueue: ExecutorPodsEventQueue) extends Logging { + podsEventQueue: ExecutorPodsEventQueue, + // Use a best-effort to track which executors have been removed already. It's not generally + // job-breaking if we remove executors more than once but it's ideal if we make an attempt + // to avoid doing so. Expire cache entries so that this data structure doesn't grow beyond + // bounds. + removedExecutorsCache: Cache[java.lang.Long, java.lang.Long] = + CacheBuilder.newBuilder() + .expireAfterWrite(3, TimeUnit.MINUTES) + .build[java.lang.Long, java.lang.Long]()) extends Logging { import ExecutorPodsLifecycleEventHandler._ - // Use a best-effort to track which executors have been removed already. It's not generally - // job-breaking if we remove executors more than once but it's ideal if we make an attempt - // to avoid doing so. Expire cache entries so that this data structure doesn't grow beyond - // bounds. - private val removedExecutors = CacheBuilder.newBuilder() - .expireAfterWrite(3, TimeUnit.MINUTES) - .build[java.lang.Long, java.lang.Long]() - def start(schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { podsEventQueue.addSubscriber(1000L) { updatedPods => updatedPods.foreach { updatedPod => @@ -82,8 +79,8 @@ private[spark] class ExecutorPodsLifecycleEventHandler( schedulerBackend: KubernetesClusterSchedulerBackend, updatedPod: Pod, execId: Long): Unit = { - if (removedExecutors.getIfPresent(execId) == null) { - removedExecutors.put(execId, execId) + if (removedExecutorsCache.getIfPresent(execId) == null) { + removedExecutorsCache.put(execId, execId) val exitReason = findExitReason(updatedPod, execId) schedulerBackend.doRemoveExecutor(execId.toString, exitReason) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 5165775dfb685..560afed94afbd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -59,14 +59,14 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( "kubernetes-executor-requests") - val eventsProcessorExecutor = ThreadUtils - .newDaemonSingleThreadScheduledExecutor("kubernetes-executor-pods-event-handler") - val eventQueue = new ExecutorPodsEventQueue(eventsProcessorExecutor) + val bufferEventsExecutor = ThreadUtils + .newDaemonSingleThreadScheduledExecutor("kubernetes-executor-pods-event-buffer") + val executeEventSubscribersExecutor = ThreadUtils + .newDaemonCachedThreadPool("kubernetes-executor-pods-event-handlers") + val eventQueue = new ExecutorPodsEventQueueImpl( + bufferEventsExecutor, executeEventSubscribersExecutor) val executorPodsLifecycleEventHandler = new ExecutorPodsLifecycleEventHandler( - sc.conf, - new KubernetesExecutorBuilder(), - kubernetesClient, - eventQueue) + new KubernetesExecutorBuilder(), kubernetesClient, eventQueue) val executorPodsAllocator = new ExecutorPodsAllocator( sc.conf, new KubernetesExecutorBuilder(), kubernetesClient, eventQueue) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index a836c57c56627..8643cbd4b56b2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -36,7 +36,7 @@ private[spark] class KubernetesClusterSchedulerBackend( podAllocator: ExecutorPodsAllocator, lifecycleEventHandler: ExecutorPodsLifecycleEventHandler, watchEvents: ExecutorPodsWatchEventSource, - pollEvents: ExecutorPodsPollingEventSource) + pollEvents: ExecutorPodsPollingEventSource) extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala new file mode 100644 index 0000000000000..dea0a0536f73e --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala @@ -0,0 +1,41 @@ +/* + * 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 io.fabric8.kubernetes.api.model.Pod +import scala.collection.mutable + +class DeterministicExecutorPodsEventQueue extends ExecutorPodsEventQueue { + + private val eventBuffer = mutable.Buffer.empty[Pod] + private val subscribers = mutable.Buffer.empty[(Seq[Pod]) => Unit] + + override def addSubscriber + (processBatchIntervalMillis: Long) + (onNextBatch: (Seq[Pod]) => Unit): Unit = { + subscribers += onNextBatch + } + + override def stopProcessingEvents(): Unit = {} + + override def pushPodUpdate(updatedPod: Pod): Unit = eventBuffer += updatedPod + + def notifySubscribers(): Unit = { + subscribers.foreach { _(eventBuffer) } + eventBuffer.clear() + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala new file mode 100644 index 0000000000000..50312f6ff7186 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala @@ -0,0 +1,91 @@ +/* + * 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 io.fabric8.kubernetes.api.model.{ContainerBuilder, Pod, PodBuilder} + +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.SparkPod + +object ExecutorLifecycleTestUtils { + + val TEST_SPARK_APP_ID = "spark-app-id" + + def failedExecutorWithoutDeletion(executorId: Int): Pod = { + new PodBuilder(podWithAttachedContainerForId(executorId)) + .editOrNewStatus() + .withPhase("error") + .addNewContainerStatus() + .withName("spark-executor") + .withImage("k8s-spark") + .withNewState() + .withNewTerminated() + .withMessage("Failed") + .withExitCode(1) + .endTerminated() + .endState() + .endContainerStatus() + .addNewContainerStatus() + .withName("spark-executor-sidecar") + .withImage("k8s-spark-sidecar") + .withNewState() + .withNewTerminated() + .withMessage("Failed") + .withExitCode(1) + .endTerminated() + .endState() + .endContainerStatus() + .withMessage("Executor failed.") + .withReason("Executor failed because of a thrown error.") + .endStatus() + .build() + } + + def runningExecutor(executorId: Int): Pod = { + new PodBuilder(podWithAttachedContainerForId(executorId)) + .editOrNewStatus() + .withPhase("running") + .endStatus() + .build() + } + + def podWithAttachedContainerForId(executorId: Int): Pod = { + val sparkPod = executorPodWithId(executorId.toString) + val podWithAttachedContainer = new PodBuilder(sparkPod.pod) + .editOrNewSpec() + .addToContainers(sparkPod.container) + .endSpec() + .build() + podWithAttachedContainer + } + + def executorPodWithId(executorId: String): SparkPod = { + val pod = new PodBuilder() + .withNewMetadata() + .withName(s"spark-executor-$executorId") + .addToLabels(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID) + .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .addToLabels(SPARK_EXECUTOR_ID_LABEL, executorId) + .endMetadata() + .build() + val container = new ContainerBuilder() + .withName("spark-executor") + .withImage("k8s-spark") + .build() + SparkPod(pod, container) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala new file mode 100644 index 0000000000000..e7c5a6b91ba7b --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -0,0 +1,163 @@ +/* + * 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 io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder, PodList} +import io.fabric8.kubernetes.client.KubernetesClient +import io.fabric8.kubernetes.client.dsl.{MixedOperation, PodResource} +import org.mockito.{ArgumentMatcher, Matchers, Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.{never, times, verify, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter +import scala.collection.mutable + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._ + +class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { + + private type Pods = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + + private val driverPodName = "driver" + + private val driverPod = new PodBuilder() + .withNewMetadata() + .withName(driverPodName) + .addToLabels(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID) + .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_DRIVER_ROLE) + .withUid("driver-pod-uid") + .endMetadata() + .build() + + private val conf = new SparkConf().set(KUBERNETES_DRIVER_POD_NAME, driverPodName) + + private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) + + private val podAllocationDelay = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) + + private var namedExecutorPods: mutable.Map[String, PodResource[Pod, DoneablePod]] = _ + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var podOperations: Pods = _ + + @Mock + private var driverPodOperations: PodResource[Pod, DoneablePod] = _ + + @Mock + private var executorBuilder: KubernetesExecutorBuilder = _ + + private var eventQueue: DeterministicExecutorPodsEventQueue = _ + + private var podsAllocatorUnderTest: ExecutorPodsAllocator = _ + + before { + MockitoAnnotations.initMocks(this) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withName(driverPodName)).thenReturn(driverPodOperations) + when(driverPodOperations.get).thenReturn(driverPod) + when(executorBuilder.buildFromFeatures(kubernetesConfWithCorrectFields())) + .thenAnswer(executorPodAnswer()) + eventQueue = new DeterministicExecutorPodsEventQueue() + podsAllocatorUnderTest = new ExecutorPodsAllocator( + conf, executorBuilder, kubernetesClient, eventQueue) + podsAllocatorUnderTest.start(TEST_SPARK_APP_ID) + } + + test("Initially request executors in batches. Do not request another batch if the" + + " first has not finished.") { + podsAllocatorUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) + eventQueue.notifySubscribers() + for (nextId <- 1 to podAllocationSize) { + verify(podOperations).create( + podWithAttachedContainerForId(nextId)) + } + verify(podOperations, never()).create( + podWithAttachedContainerForId(podAllocationSize + 1)) + } + + test("Request executors in batches. Allow another batch to be requested if" + + " all pending executors start running.") { + podsAllocatorUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) + eventQueue.notifySubscribers() + for (execId <- 1 until podAllocationSize) { + eventQueue.pushPodUpdate(runningExecutor(execId)) + } + eventQueue.notifySubscribers() + verify(podOperations, never()).create( + podWithAttachedContainerForId(podAllocationSize + 1)) + eventQueue.pushPodUpdate( + runningExecutor(podAllocationSize)) + eventQueue.notifySubscribers() + verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) + eventQueue.notifySubscribers() + verify(podOperations, times(podAllocationSize + 1)).create(any(classOf[Pod])) + } + + test("When a current batch reaches error states immediately, re-request" + + " them on the next batch.") { + podsAllocatorUnderTest.setTotalExpectedExecutors(podAllocationSize) + eventQueue.notifySubscribers() + for (execId <- 1 until podAllocationSize) { + eventQueue.pushPodUpdate(runningExecutor(execId)) + } + val failedPod = failedExecutorWithoutDeletion(podAllocationSize) + eventQueue.pushPodUpdate(failedPod) + eventQueue.notifySubscribers() + verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) + } + + private def executorPodAnswer(): Answer[SparkPod] = { + new Answer[SparkPod] { + override def answer(invocation: InvocationOnMock): SparkPod = { + val k8sConf = invocation.getArgumentAt( + 0, classOf[KubernetesConf[KubernetesExecutorSpecificConf]]) + executorPodWithId(k8sConf.roleSpecificConf.executorId) + } + } + } + + private def kubernetesConfWithCorrectFields(): KubernetesConf[KubernetesExecutorSpecificConf] = + Matchers.argThat(new ArgumentMatcher[KubernetesConf[KubernetesExecutorSpecificConf]] { + override def matches(argument: scala.Any): Boolean = { + if (!argument.isInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]]) { + false + } else { + val k8sConf = argument.asInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]] + val executorSpecificConf = k8sConf.roleSpecificConf + val expectedK8sConf = KubernetesConf.createExecutorConf( + conf, + executorSpecificConf.executorId, + TEST_SPARK_APP_ID, + driverPod) + k8sConf.sparkConf.getAll.toMap == conf.getAll.toMap && + // Since KubernetesConf.createExecutorConf clones the SparkConf object, force + // deep equality comparison for the SparkConf object and use object equality + // comparison on all other fields. + k8sConf.copy(sparkConf = conf) == expectedK8sConf.copy(sparkConf = conf) + } + } + }) + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala deleted file mode 100644 index a7c3095308c66..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventHandlerSuite.scala +++ /dev/null @@ -1,269 +0,0 @@ -/* - * 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 org.scalatest.BeforeAndAfter - -import org.apache.spark.SparkFunSuite - -class ExecutorPodsEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { - - // TODO - /* - private type Pods = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] - - private val driverPodName = "driver" - - private val appId = "spark" - - private val driverPod = new PodBuilder() - .withNewMetadata() - .withName(driverPodName) - .addToLabels(SPARK_APP_ID_LABEL, appId) - .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_DRIVER_ROLE) - .withUid("driver-pod-uid") - .endMetadata() - .build() - - private val conf = new SparkConf().set(KUBERNETES_DRIVER_POD_NAME, driverPodName) - - private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) - - private val podAllocationDelay = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) - - private val eventProcessorExecutor = new DeterministicScheduler - - private var namedExecutorPods: mutable.Map[String, PodResource[Pod, DoneablePod]] = _ - - @Mock - private var kubernetesClient: KubernetesClient = _ - - @Mock - private var podOperations: Pods = _ - - @Mock - private var driverPodOperations: PodResource[Pod, DoneablePod] = _ - - @Mock - private var executorBuilder: KubernetesExecutorBuilder = _ - - @Mock - private var schedulerBackend: KubernetesClusterSchedulerBackend = _ - - private var eventHandlerUnderTest: ExecutorPodsLifecycleEventHandler = _ - - before { - MockitoAnnotations.initMocks(this) - namedExecutorPods = mutable.Map.empty[String, PodResource[Pod, DoneablePod]] - when(kubernetesClient.pods()).thenReturn(podOperations) - when(podOperations.withName(any(classOf[String]))).thenAnswer(namedPodsAnswer()) - when(podOperations.withName(driverPodName)).thenReturn(driverPodOperations) - when(driverPodOperations.get).thenReturn(driverPod) - when(executorBuilder.buildFromFeatures(kubernetesConfWithCorrectFields())) - .thenAnswer(executorPodAnswer()) - eventHandlerUnderTest = new ExecutorPodsLifecycleEventHandler( - conf, executorBuilder, kubernetesClient, eventProcessorExecutor) - eventHandlerUnderTest.start(appId, schedulerBackend) - } - - test("Initially request executors in batches. Do not request another batch if the" + - " first has not finished.") { - eventHandlerUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) - runProcessor() - for (nextId <- 1 to podAllocationSize) { - verify(podOperations).create(podWithAttachedContainerForId(nextId)) - } - verify(podOperations, never()).create( - podWithAttachedContainerForId(podAllocationSize + 1)) - } - - test("Request executors in batches. Allow another batch to be requested if" + - " all pending executors start running.") { - eventHandlerUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) - runProcessor() - for (execId <- 1 until podAllocationSize) { - eventHandlerUnderTest.sendUpdatedPodMetadata(runningExecutor(execId)) - } - runProcessor() - verify(podOperations, never()).create( - podWithAttachedContainerForId(podAllocationSize + 1)) - eventHandlerUnderTest.sendUpdatedPodMetadata(runningExecutor(podAllocationSize)) - runProcessor() - verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) - runProcessor() - verify(podOperations, times(podAllocationSize + 1)).create(any(classOf[Pod])) - } - - test("When a current batch reaches error states immediately, re-request" + - " them on the next batch.") { - eventHandlerUnderTest.setTotalExpectedExecutors(podAllocationSize) - runProcessor() - for (execId <- 1 until podAllocationSize) { - eventHandlerUnderTest.sendUpdatedPodMetadata(runningExecutor(execId)) - } - val failedPod = failedExecutorWithoutDeletion(podAllocationSize) - eventHandlerUnderTest.sendUpdatedPodMetadata(failedPod) - runProcessor() - val msg = exitReasonMessage(podAllocationSize, failedPod) - val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) - verify(schedulerBackend).doRemoveExecutor(podAllocationSize.toString, expectedLossReason) - verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) - verify(namedExecutorPods(failedPod.getMetadata.getName)).delete() - } - - test("When a current batch reaches a running state and then one executor reaches an error" + - " state, re-request it on the next batch.") { - eventHandlerUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) - runProcessor() - for (execId <- 1 to podAllocationSize) { - eventHandlerUnderTest.sendUpdatedPodMetadata(runningExecutor(execId)) - } - runProcessor() - eventHandlerUnderTest.sendUpdatedPodMetadata(runningExecutor(podAllocationSize + 1)) - val failedExecutorId = podAllocationSize - 1 - val failedPod = failedExecutorWithoutDeletion(failedExecutorId) - eventHandlerUnderTest.sendUpdatedPodMetadata(failedPod) - runProcessor() - val msg = exitReasonMessage(failedExecutorId, failedPod) - val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) - verify(schedulerBackend).doRemoveExecutor(failedExecutorId.toString, expectedLossReason) - verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 2)) - verify(namedExecutorPods(failedPod.getMetadata.getName)).delete() - } - - private def exitReasonMessage(failedExecutorId: Int, failedPod: Pod): String = { - s""" - |The executor with id $failedExecutorId exited with exit code 1. - |The API gave the following brief reason: ${failedPod.getStatus.getReason} - |The API gave the following message: ${failedPod.getStatus.getMessage} - |The API gave the following container statuses: - | - |${failedPod.getStatus.getContainerStatuses.asScala.map(_.toString).mkString("\n===\n")} - """.stripMargin - } - - private def runProcessor(): Unit = { - eventProcessorExecutor.tick(podAllocationDelay, TimeUnit.MILLISECONDS) - } - - private def runningExecutor(executorId: Int): Pod = { - new PodBuilder(podWithAttachedContainerForId(executorId)) - .editOrNewStatus() - .withPhase("running") - .endStatus() - .build() - } - - private def failedExecutorWithoutDeletion(executorId: Int): Pod = { - new PodBuilder(podWithAttachedContainerForId(executorId)) - .editOrNewStatus() - .withPhase("error") - .addNewContainerStatus() - .withName("spark-executor") - .withImage("k8s-spark") - .withNewState() - .withNewTerminated() - .withMessage("Failed") - .withExitCode(1) - .endTerminated() - .endState() - .endContainerStatus() - .addNewContainerStatus() - .withName("spark-executor-sidecar") - .withImage("k8s-spark-sidecar") - .withNewState() - .withNewTerminated() - .withMessage("Failed") - .withExitCode(1) - .endTerminated() - .endState() - .endContainerStatus() - .withMessage("Executor failed.") - .withReason("Executor failed because of a thrown error.") - .endStatus() - .build() - } - - private def kubernetesConfWithCorrectFields(): KubernetesConf[KubernetesExecutorSpecificConf] = - Matchers.argThat(new ArgumentMatcher[KubernetesConf[KubernetesExecutorSpecificConf]] { - override def matches(argument: scala.Any): Boolean = { - if (!argument.isInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]]) { - false - } else { - val k8sConf = argument.asInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]] - val executorSpecificConf = k8sConf.roleSpecificConf - val expectedK8sConf = KubernetesConf.createExecutorConf( - conf, - executorSpecificConf.executorId, - appId, - driverPod) - k8sConf.sparkConf.getAll.toMap == conf.getAll.toMap && - // Since KubernetesConf.createExecutorConf clones the SparkConf object, force - // deep equality comparison for the SparkConf object and use object equality - // comparison on all other fields. - k8sConf.copy(sparkConf = conf) == expectedK8sConf.copy(sparkConf = conf) - } - } - }) - - private def executorPodAnswer(): Answer[SparkPod] = { - new Answer[SparkPod] { - override def answer(invocation: InvocationOnMock): SparkPod = { - val k8sConf = invocation.getArgumentAt( - 0, classOf[KubernetesConf[KubernetesExecutorSpecificConf]]) - executorPodWithId(k8sConf.roleSpecificConf.executorId) - } - } - } - - private def namedPodsAnswer(): Answer[PodResource[Pod, DoneablePod]] = { - new Answer[PodResource[Pod, DoneablePod]] { - override def answer(invocation: InvocationOnMock): PodResource[Pod, DoneablePod] = { - val podName = invocation.getArgumentAt(0, classOf[String]) - namedExecutorPods.getOrElseUpdate( - podName, mock(classOf[PodResource[Pod, DoneablePod]])) - } - } - } - - private def podWithAttachedContainerForId(executorId: Int): Pod = { - val sparkPod = executorPodWithId(executorId.toString) - val podWithAttachedContainer = new PodBuilder(sparkPod.pod) - .editOrNewSpec() - .addToContainers(sparkPod.container) - .endSpec() - .build() - podWithAttachedContainer - } - - private def executorPodWithId(executorId: String): SparkPod = { - val pod = new PodBuilder() - .withNewMetadata() - .withName(s"spark-executor-$executorId") - .addToLabels(SPARK_APP_ID_LABEL, appId) - .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) - .addToLabels(SPARK_EXECUTOR_ID_LABEL, executorId) - .endMetadata() - .build() - val container = new ContainerBuilder() - .withName("spark-executor") - .withImage("k8s-spark") - .build() - SparkPod(pod, container) - } - */ -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala index 2efc20f447afa..9c8de20a58095 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala @@ -20,7 +20,7 @@ import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicReference import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} -import org.jmock.lib.concurrent.DeterministicScheduler +import org.jmock.lib.concurrent.{DeterministicExecutor, DeterministicScheduler} import org.scalatest.BeforeAndAfter import scala.collection.mutable @@ -28,12 +28,16 @@ import org.apache.spark.SparkFunSuite class ExecutorPodsEventQueueSuite extends SparkFunSuite with BeforeAndAfter { - private var eventProcessor: DeterministicScheduler = _ - private var eventQueueUnderTest: ExecutorPodsEventQueue = _ + private var eventBufferScheduler: DeterministicScheduler = _ + private var executeSubscriptionsExecutor: DeterministicExecutor = _ + private var eventQueueUnderTest: ExecutorPodsEventQueueImpl = _ before { - eventProcessor = new DeterministicScheduler() - eventQueueUnderTest = new ExecutorPodsEventQueue(eventProcessor) + eventBufferScheduler = new DeterministicScheduler() + executeSubscriptionsExecutor = new DeterministicExecutor + eventQueueUnderTest = new ExecutorPodsEventQueueImpl( + eventBufferScheduler, + executeSubscriptionsExecutor) } test("Subscribers get notified of events periodically.") { @@ -45,17 +49,24 @@ class ExecutorPodsEventQueueSuite extends SparkFunSuite with BeforeAndAfter { pushPodWithIndex(1) assert(receivedEvents1.isEmpty) assert(receivedEvents2.isEmpty) - eventProcessor.tick(1000, TimeUnit.MILLISECONDS) + // Force time to move forward so that the buffer is emitted, scheduling the + // processing task on the subscription executor... + eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) + // ... then actually execute the subscribers. + executeSubscriptionsExecutor.runUntilIdle() assertIndicesMatch(receivedEvents1, 1) assert(receivedEvents2.isEmpty) - eventProcessor.tick(1000, TimeUnit.MILLISECONDS) + eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) + executeSubscriptionsExecutor.runUntilIdle() assertIndicesMatch(receivedEvents2, 1) pushPodWithIndex(2) pushPodWithIndex(3) - eventProcessor.tick(1000, TimeUnit.MILLISECONDS) + eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) + executeSubscriptionsExecutor.runUntilIdle() assertIndicesMatch(receivedEvents1, 1, 2, 3) assertIndicesMatch(receivedEvents2, 1) - eventProcessor.tick(2000, TimeUnit.MILLISECONDS) + eventBufferScheduler.tick(2000, TimeUnit.MILLISECONDS) + executeSubscriptionsExecutor.runUntilIdle() assertIndicesMatch(receivedEvents1, 1, 2, 3) assertIndicesMatch(receivedEvents2, 1, 2, 3) } @@ -63,6 +74,8 @@ class ExecutorPodsEventQueueSuite extends SparkFunSuite with BeforeAndAfter { test("Even without sending events, initially receive an empty buffer.") { val receivedInitialBuffer = new AtomicReference[Seq[Pod]](null) eventQueueUnderTest.addSubscriber(1000) { receivedInitialBuffer.set } + assert(receivedInitialBuffer.get == null) + executeSubscriptionsExecutor.runPendingCommands() assert(receivedInitialBuffer.get != null) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala new file mode 100644 index 0000000000000..99ced3c69969e --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.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.scheduler.cluster.k8s + +import com.google.common.cache.CacheBuilder +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodList} +import io.fabric8.kubernetes.client.KubernetesClient +import io.fabric8.kubernetes.client.dsl.{MixedOperation, PodResource} +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.{mock, times, verify, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.SparkFunSuite +import org.apache.spark.scheduler.ExecutorExited +import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._ + +class ExecutorPodsLifecycleEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { + + private type Pods = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + + private var namedExecutorPods: mutable.Map[String, PodResource[Pod, DoneablePod]] = _ + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var podOperations: Pods = _ + + @Mock + private var driverPodOperations: PodResource[Pod, DoneablePod] = _ + + @Mock + private var executorBuilder: KubernetesExecutorBuilder = _ + + @Mock + private var schedulerBackend: KubernetesClusterSchedulerBackend = _ + + private var eventQueue: DeterministicExecutorPodsEventQueue = _ + private var eventHandlerUnderTest: ExecutorPodsLifecycleEventHandler = _ + + before { + MockitoAnnotations.initMocks(this) + val removedExecutorsCache = CacheBuilder.newBuilder().build[java.lang.Long, java.lang.Long] + eventQueue = new DeterministicExecutorPodsEventQueue() + namedExecutorPods = mutable.Map.empty[String, PodResource[Pod, DoneablePod]] + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withName(any(classOf[String]))).thenAnswer(namedPodsAnswer()) + eventHandlerUnderTest = new ExecutorPodsLifecycleEventHandler( + executorBuilder, kubernetesClient, eventQueue, removedExecutorsCache) + eventHandlerUnderTest.start(schedulerBackend) + } + + test("When an executor reaches error states immediately, remove from the scheduler backend.") { + val failedPod = failedExecutorWithoutDeletion(1) + eventQueue.pushPodUpdate(failedPod) + eventQueue.notifySubscribers() + val msg = exitReasonMessage(1, failedPod) + val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) + verify(schedulerBackend).doRemoveExecutor("1", expectedLossReason) + verify(namedExecutorPods(failedPod.getMetadata.getName)).delete() + } + + test("Don't remove executors twice from Spark but remove from K8s repeatedly.") { + val failedPod = failedExecutorWithoutDeletion(1) + eventQueue.pushPodUpdate(failedPod) + eventQueue.pushPodUpdate(failedPod) + eventQueue.notifySubscribers() + val msg = exitReasonMessage(1, failedPod) + val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) + verify(schedulerBackend, times(1)).doRemoveExecutor("1", expectedLossReason) + verify(namedExecutorPods(failedPod.getMetadata.getName), times(2)).delete() + } + + private def exitReasonMessage(failedExecutorId: Int, failedPod: Pod): String = { + s""" + |The executor with id $failedExecutorId exited with exit code 1. + |The API gave the following brief reason: ${failedPod.getStatus.getReason} + |The API gave the following message: ${failedPod.getStatus.getMessage} + |The API gave the following container statuses: + | + |${failedPod.getStatus.getContainerStatuses.asScala.map(_.toString).mkString("\n===\n")} + """.stripMargin + } + + private def namedPodsAnswer(): Answer[PodResource[Pod, DoneablePod]] = { + new Answer[PodResource[Pod, DoneablePod]] { + override def answer(invocation: InvocationOnMock): PodResource[Pod, DoneablePod] = { + val podName = invocation.getArgumentAt(0, classOf[String]) + namedExecutorPods.getOrElseUpdate( + podName, mock(classOf[PodResource[Pod, DoneablePod]])) + } + } + } +} From c398ebbe71e3ca586961df8fa2033b15235b27c2 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 24 May 2018 18:02:09 -0700 Subject: [PATCH 22/47] Add more tests. --- .../org/apache/spark/deploy/k8s/Config.scala | 18 +++ .../ExecutorPodsLifecycleEventHandler.scala | 7 +- .../k8s/ExecutorPodsPollingEventSource.scala | 9 +- .../k8s/ExecutorPodsWatchEventSource.scala | 2 +- .../k8s/KubernetesClusterManager.scala | 7 +- .../KubernetesClusterSchedulerBackend.scala | 4 +- .../spark/deploy/k8s/Fabric8Aliases.scala | 30 ++++ .../spark/deploy/k8s/submit/ClientSuite.scala | 9 +- .../k8s/ExecutorPodsAllocatorSuite.scala | 5 +- ...ecutorPodsLifecycleEventHandlerSuite.scala | 15 +- .../ExecutorPodsPollingEventSourceSuite.scala | 86 ++++++++++ .../ExecutorPodsWatchEventSourceSuite.scala | 75 +++++++++ ...bernetesClusterSchedulerBackendSuite.scala | 152 ++++++++++++++++++ 13 files changed, 397 insertions(+), 22 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/Fabric8Aliases.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 4086970ffb256..a4279dc6fea2c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -154,6 +154,24 @@ private[spark] object Config extends Logging { .checkValue(interval => interval > 0, s"Logging interval must be a positive time value.") .createWithDefaultString("1s") + val KUBERNETES_EXECUTOR_API_POLLING_INTERVAL = + ConfigBuilder("spark.kubernetes.executor.apiPollingInterval") + .doc("Interval between polls against the Kubernetes API server to inspect the" + + " state of executors.") + .timeConf(TimeUnit.MILLISECONDS) + .checkValue(interval => interval > 0, s"API server polling interval must be a" + + s" positive time value.") + .createWithDefaultString("30s") + + val KUBERNETES_EXECUTOR_EVENT_PROCESSING_INTERVAL = + ConfigBuilder("spark.kubernetes.executor.eventProcessingInterval") + .doc("Interval between successive inspection of executor events sent from the" + + " Kubernetes API.") + .timeConf(TimeUnit.MILLISECONDS) + .checkValue(interval => interval > 0, s"Event processing interval must be a positive" + + s" time value.") + .createWithDefaultString("1s") + val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = "spark.kubernetes.authenticate.submission" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala index beb758466e789..5b36cbdf320c6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala @@ -23,12 +23,15 @@ import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.JavaConverters._ +import org.apache.spark.SparkConf +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.ExecutorExited import org.apache.spark.util.Utils private[spark] class ExecutorPodsLifecycleEventHandler( + conf: SparkConf, executorBuilder: KubernetesExecutorBuilder, kubernetesClient: KubernetesClient, podsEventQueue: ExecutorPodsEventQueue, @@ -43,8 +46,10 @@ private[spark] class ExecutorPodsLifecycleEventHandler( import ExecutorPodsLifecycleEventHandler._ + private val eventProcessingInterval = conf.get(KUBERNETES_EXECUTOR_EVENT_PROCESSING_INTERVAL) + def start(schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { - podsEventQueue.addSubscriber(1000L) { updatedPods => + podsEventQueue.addSubscriber(eventProcessingInterval) { updatedPods => updatedPods.foreach { updatedPod => processUpdatedPod(schedulerBackend, updatedPod) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala index 7c9b2b04a0f35..0da213b9d8ff5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala @@ -21,18 +21,24 @@ import java.util.concurrent.{Future, ScheduledExecutorService, TimeUnit} import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.JavaConverters._ +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ private[spark] class ExecutorPodsPollingEventSource( + conf: SparkConf, kubernetesClient: KubernetesClient, eventQueue: ExecutorPodsEventQueue, pollingExecutor: ScheduledExecutorService) { + + private val pollingInterval = conf.get(KUBERNETES_EXECUTOR_API_POLLING_INTERVAL) + private var pollingFuture: Future[_] = _ def start(applicationId: String): Unit = { require(pollingFuture == null, "Cannot start polling more than once.") pollingFuture = pollingExecutor.scheduleWithFixedDelay( - new PollRunnable(applicationId), 0L, 30L, TimeUnit.SECONDS) + new PollRunnable(applicationId), pollingInterval, pollingInterval, TimeUnit.MILLISECONDS) } def stop(): Unit = { @@ -40,6 +46,7 @@ private[spark] class ExecutorPodsPollingEventSource( pollingFuture.cancel(true) pollingFuture = null } + pollingExecutor.shutdown() } private class PollRunnable(applicationId: String) extends Runnable { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala index 288ea326c4af7..377b3bbc017df 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala @@ -36,7 +36,7 @@ private[spark] class ExecutorPodsWatchEventSource( require(watchConnection == null, "Cannot start the watcher twice.") watchConnection = kubernetesClient.pods() .withLabel(SPARK_APP_ID_LABEL, applicationId) - .withLabel(SPARK_POD_EXECUTOR_ROLE) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .watch(new ExecutorPodsWatcher()) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 560afed94afbd..c44ad092d681d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -66,7 +66,10 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val eventQueue = new ExecutorPodsEventQueueImpl( bufferEventsExecutor, executeEventSubscribersExecutor) val executorPodsLifecycleEventHandler = new ExecutorPodsLifecycleEventHandler( - new KubernetesExecutorBuilder(), kubernetesClient, eventQueue) + sc.conf, + new KubernetesExecutorBuilder(), + kubernetesClient, + eventQueue) val executorPodsAllocator = new ExecutorPodsAllocator( sc.conf, new KubernetesExecutorBuilder(), kubernetesClient, eventQueue) @@ -78,7 +81,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val eventsPollingExecutor = ThreadUtils.newDaemonSingleThreadScheduledExecutor( "kubernetes-executor-pod-polling-sync") val podsPollingEventSource = new ExecutorPodsPollingEventSource( - kubernetesClient, eventQueue, eventsPollingExecutor) + sc.conf, kubernetesClient, eventQueue, eventsPollingExecutor) new KubernetesClusterSchedulerBackend( scheduler.asInstanceOf[TaskSchedulerImpl], diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 8643cbd4b56b2..3dee474fb0fc3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -36,7 +36,7 @@ private[spark] class KubernetesClusterSchedulerBackend( podAllocator: ExecutorPodsAllocator, lifecycleEventHandler: ExecutorPodsLifecycleEventHandler, watchEvents: ExecutorPodsWatchEventSource, - pollEvents: ExecutorPodsPollingEventSource) + pollEvents: ExecutorPodsPollingEventSource) extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( @@ -51,7 +51,7 @@ private[spark] class KubernetesClusterSchedulerBackend( private val initialExecutors = SchedulerBackendUtils.getInitialTargetExecutorNumber(conf) - // Allow removeExecutor to be accessible by KubernetesSchedulerEventHandler + // Allow removeExecutor to be accessible by ExecutorPodsLifecycleEventHandler private[k8s] def doRemoveExecutor(executorId: String, reason: ExecutorLossReason): Unit = { removeExecutor(executorId, reason) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/Fabric8Aliases.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/Fabric8Aliases.scala new file mode 100644 index 0000000000000..527fc6b0d8f87 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/Fabric8Aliases.scala @@ -0,0 +1,30 @@ +/* + * 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 io.fabric8.kubernetes.api.model.{DoneablePod, HasMetadata, Pod, PodList} +import io.fabric8.kubernetes.client.{Watch, Watcher} +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, PodResource} + +object Fabric8Aliases { + type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + type LABELED_PODS = FilterWatchListDeletable[ + Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]] + type SINGLE_POD = PodResource[Pod, DoneablePod] + type RESOURCE_LIST = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[ + HasMetadata, Boolean] +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala index c1b203e03a357..959570985b20f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -27,6 +27,7 @@ import org.scalatest.mockito.MockitoSugar._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.Fabric8Aliases._ class ClientSuite extends SparkFunSuite with BeforeAndAfter { @@ -103,15 +104,11 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { .build() } - private type ResourceList = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[ - HasMetadata, Boolean] - private type Pods = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] - @Mock private var kubernetesClient: KubernetesClient = _ @Mock - private var podOperations: Pods = _ + private var podOperations: PODS = _ @Mock private var namedPods: PodResource[Pod, DoneablePod] = _ @@ -123,7 +120,7 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { private var driverBuilder: KubernetesDriverBuilder = _ @Mock - private var resourceList: ResourceList = _ + private var resourceList: RESOURCE_LIST = _ private var kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf] = _ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index e7c5a6b91ba7b..2c46147236670 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -31,12 +31,11 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.Fabric8Aliases._ import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { - private type Pods = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] - private val driverPodName = "driver" private val driverPod = new PodBuilder() @@ -60,7 +59,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { private var kubernetesClient: KubernetesClient = _ @Mock - private var podOperations: Pods = _ + private var podOperations: PODS = _ @Mock private var driverPodOperations: PodResource[Pod, DoneablePod] = _ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala index 99ced3c69969e..a8df695e9abf2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala @@ -29,24 +29,23 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Fabric8Aliases._ import org.apache.spark.scheduler.ExecutorExited import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._ class ExecutorPodsLifecycleEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { - private type Pods = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] - private var namedExecutorPods: mutable.Map[String, PodResource[Pod, DoneablePod]] = _ @Mock private var kubernetesClient: KubernetesClient = _ @Mock - private var podOperations: Pods = _ + private var podOperations: PODS = _ @Mock - private var driverPodOperations: PodResource[Pod, DoneablePod] = _ + private var driverPodOperations: SINGLE_POD = _ @Mock private var executorBuilder: KubernetesExecutorBuilder = _ @@ -65,7 +64,11 @@ class ExecutorPodsLifecycleEventHandlerSuite extends SparkFunSuite with BeforeAn when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(any(classOf[String]))).thenAnswer(namedPodsAnswer()) eventHandlerUnderTest = new ExecutorPodsLifecycleEventHandler( - executorBuilder, kubernetesClient, eventQueue, removedExecutorsCache) + new SparkConf(), + executorBuilder, + kubernetesClient, + eventQueue, + removedExecutorsCache) eventHandlerUnderTest.start(schedulerBackend) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala new file mode 100644 index 0000000000000..644ce7b018e21 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala @@ -0,0 +1,86 @@ +/* + * 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.TimeUnit + +import io.fabric8.kubernetes.api.model.PodListBuilder +import io.fabric8.kubernetes.client.KubernetesClient +import org.jmock.lib.concurrent.DeterministicScheduler +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Mockito.{verify, when} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.Fabric8Aliases._ +import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._ + +class ExecutorPodsPollingEventSourceSuite extends SparkFunSuite with BeforeAndAfter { + + private val sparkConf = new SparkConf + + private val pollingInterval = sparkConf.get(KUBERNETES_EXECUTOR_API_POLLING_INTERVAL) + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var podOperations: PODS = _ + + @Mock + private var appIdLabeledPods: LABELED_PODS = _ + + @Mock + private var executorRoleLabeledPods: LABELED_PODS = _ + + @Mock + private var eventQueue: ExecutorPodsEventQueue = _ + + private var pollingExecutor: DeterministicScheduler = _ + private var pollingSourceUnderTest: ExecutorPodsPollingEventSource = _ + + before { + MockitoAnnotations.initMocks(this) + pollingExecutor = new DeterministicScheduler() + pollingSourceUnderTest = new ExecutorPodsPollingEventSource( + sparkConf, + kubernetesClient, + eventQueue, + pollingExecutor) + pollingSourceUnderTest.start(TEST_SPARK_APP_ID) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)) + .thenReturn(appIdLabeledPods) + when(appIdLabeledPods.withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) + .thenReturn(executorRoleLabeledPods) + } + + test("Items returned by the API should be pushed to the event queue") { + when(executorRoleLabeledPods.list()) + .thenReturn(new PodListBuilder() + .addToItems( + runningExecutor(1), + runningExecutor(2)) + .build()) + pollingExecutor.tick(pollingInterval, TimeUnit.MILLISECONDS) + verify(eventQueue).pushPodUpdate(runningExecutor(1)) + verify(eventQueue).pushPodUpdate(runningExecutor(2)) + + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala new file mode 100644 index 0000000000000..465ab845e09e7 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala @@ -0,0 +1,75 @@ +/* + * 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 io.fabric8.kubernetes.api.model.Pod +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} +import org.mockito.Mockito.{verify, when} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.Fabric8Aliases._ +import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._ + +class ExecutorPodsWatchEventSourceSuite extends SparkFunSuite with BeforeAndAfter { + + @Mock + private var eventQueue: ExecutorPodsEventQueue = _ + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var podOperations: PODS = _ + + @Mock + private var appIdLabeledPods: LABELED_PODS = _ + + @Mock + private var executorRoleLabeledPods: LABELED_PODS = _ + + @Mock + private var watchConnection: Watch = _ + + private var watch: ArgumentCaptor[Watcher[Pod]] = _ + + private var watchEventSourceUnderTest: ExecutorPodsWatchEventSource = _ + + before { + MockitoAnnotations.initMocks(this) + watch = ArgumentCaptor.forClass(classOf[Watcher[Pod]]) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)) + .thenReturn(appIdLabeledPods) + when(appIdLabeledPods.withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) + .thenReturn(executorRoleLabeledPods) + when(executorRoleLabeledPods.watch(watch.capture())).thenReturn(watchConnection) + watchEventSourceUnderTest = new ExecutorPodsWatchEventSource( + eventQueue, kubernetesClient) + watchEventSourceUnderTest.start(TEST_SPARK_APP_ID) + } + + test("Watch events should be pushed to the queue.") { + watch.getValue.eventReceived(Action.ADDED, runningExecutor(1)) + watch.getValue.eventReceived(Action.MODIFIED, runningExecutor(2)) + verify(eventQueue).pushPodUpdate(runningExecutor(1)) + verify(eventQueue).pushPodUpdate(runningExecutor(2)) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala new file mode 100644 index 0000000000000..29c99c80d7b33 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -0,0 +1,152 @@ +/* + * 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 io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodList} +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher} +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, PodResource} +import org.jmock.lib.concurrent.DeterministicScheduler +import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} +import org.mockito.Matchers.{eq => mockitoEq} +import org.mockito.Mockito.{never, verify, when} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.Fabric8Aliases._ +import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} +import org.apache.spark.scheduler.{ExecutorKilled, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils.TEST_SPARK_APP_ID + +class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAndAfter { + + private val requestExecutorsService = new DeterministicScheduler() + private val sparkConf = new SparkConf(false) + .set("spark.executor.instances", "3") + + @Mock + private var sc: SparkContext = _ + + @Mock + private var rpcEnv: RpcEnv = _ + + @Mock + private var driverEndpointRef: RpcEndpointRef = _ + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var podOperations: PODS = _ + + @Mock + private var labeledPods: LABELED_PODS = _ + + @Mock + private var taskScheduler: TaskSchedulerImpl = _ + + @Mock + private var eventQueue: ExecutorPodsEventQueue = _ + + @Mock + private var podAllocator: ExecutorPodsAllocator = _ + + @Mock + private var lifecycleEventHandler: ExecutorPodsLifecycleEventHandler = _ + + @Mock + private var watchEvents: ExecutorPodsWatchEventSource = _ + + @Mock + private var pollEvents: ExecutorPodsPollingEventSource = _ + + private var driverEndpoint: ArgumentCaptor[RpcEndpoint] = _ + private var schedulerBackendUnderTest: KubernetesClusterSchedulerBackend = _ + + before { + MockitoAnnotations.initMocks(this) + when(taskScheduler.sc).thenReturn(sc) + when(sc.conf).thenReturn(sparkConf) + driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint]) + when(rpcEnv.setupEndpoint( + mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture())) + .thenReturn(driverEndpointRef) + when(kubernetesClient.pods()).thenReturn(podOperations) + schedulerBackendUnderTest = new KubernetesClusterSchedulerBackend( + taskScheduler, + rpcEnv, + kubernetesClient, + requestExecutorsService, + eventQueue, + podAllocator, + lifecycleEventHandler, + watchEvents, + pollEvents) { + override def applicationId(): String = TEST_SPARK_APP_ID + } + } + + test("Start all components") { + schedulerBackendUnderTest.start() + verify(podAllocator).setTotalExpectedExecutors(3) + verify(podAllocator).start(TEST_SPARK_APP_ID) + verify(lifecycleEventHandler).start(schedulerBackendUnderTest) + verify(watchEvents).start(TEST_SPARK_APP_ID) + verify(pollEvents).start(TEST_SPARK_APP_ID) + } + + test("Stop all components") { + when(podOperations.withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)).thenReturn(labeledPods) + when(labeledPods.withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)).thenReturn(labeledPods) + schedulerBackendUnderTest.stop() + verify(eventQueue).stopProcessingEvents() + verify(watchEvents).stop() + verify(pollEvents).stop() + verify(labeledPods).delete() + verify(kubernetesClient).close() + } + + test("Remove executor") { + schedulerBackendUnderTest.start() + schedulerBackendUnderTest.doRemoveExecutor( + "1", ExecutorKilled) + verify(driverEndpointRef).send(RemoveExecutor("1", ExecutorKilled)) + } + + test("Kill executors") { + schedulerBackendUnderTest.start() + when(podOperations.withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)).thenReturn(labeledPods) + when(labeledPods.withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)).thenReturn(labeledPods) + when(labeledPods.withLabelIn(SPARK_EXECUTOR_ID_LABEL, "1", "2")).thenReturn(labeledPods) + schedulerBackendUnderTest.doKillExecutors(Seq("1", "2")) + verify(labeledPods, never()).delete() + requestExecutorsService.runNextPendingCommand() + verify(labeledPods).delete() + } + + test("Request total executors") { + schedulerBackendUnderTest.start() + schedulerBackendUnderTest.doRequestTotalExecutors(5) + verify(podAllocator).setTotalExpectedExecutors(3) + verify(podAllocator, never()).setTotalExpectedExecutors(5) + requestExecutorsService.runNextPendingCommand() + verify(podAllocator).setTotalExpectedExecutors(5) + } + +} From 45a02de19a07217084caaa0a5d87b424e1b79d2e Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 24 May 2018 18:10:06 -0700 Subject: [PATCH 23/47] Minor style --- .../scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala index 377b3bbc017df..778574574ad81 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala @@ -30,7 +30,7 @@ private[spark] class ExecutorPodsWatchEventSource( podsEventQueue: ExecutorPodsEventQueue, kubernetesClient: KubernetesClient) extends Logging { - private var watchConnection: Closeable = null + private var watchConnection: Closeable = _ def start(applicationId: String): Unit = { require(watchConnection == null, "Cannot start the watcher twice.") From a8a35394ec2542c1f98c67ceef4c19b88477098e Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 25 May 2018 07:40:49 -0700 Subject: [PATCH 24/47] Minor style --- resource-managers/kubernetes/core/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index def7be4060b97..55c15c952c938 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -83,7 +83,7 @@ 3.8.1 - + io.reactivex.rxjava2 rxjava From 4a496777f7f8bfb8bb2b78a87adb626d909bfd69 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 25 May 2018 10:57:47 -0700 Subject: [PATCH 25/47] Address comments. --- .../main/scala/org/apache/spark/deploy/k8s/Config.scala | 4 ++-- .../scheduler/cluster/k8s/ExecutorPodsEventQueue.scala | 4 ++-- .../cluster/k8s/ExecutorPodsEventQueueImpl.scala | 4 ++-- .../cluster/k8s/ExecutorPodsLifecycleEventHandler.scala | 5 +---- .../cluster/k8s/ExecutorPodsPollingEventSource.scala | 2 +- .../cluster/k8s/ExecutorPodsWatchEventSource.scala | 2 +- .../scheduler/cluster/k8s/KubernetesClusterManager.scala | 8 +++++++- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 5 ++++- .../cluster/k8s/DeterministicExecutorPodsEventQueue.scala | 4 ++-- .../cluster/k8s/ExecutorPodsAllocatorSuite.scala | 8 ++++---- .../cluster/k8s/ExecutorPodsEventQueueSuite.scala | 2 +- .../k8s/ExecutorPodsLifecycleEventHandlerSuite.scala | 6 +++--- .../cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala | 4 ++-- .../cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala | 4 ++-- .../k8s/KubernetesClusterSchedulerBackendSuite.scala | 2 +- 15 files changed, 35 insertions(+), 29 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index a4279dc6fea2c..351ec30e6b68c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -156,8 +156,8 @@ private[spark] object Config extends Logging { val KUBERNETES_EXECUTOR_API_POLLING_INTERVAL = ConfigBuilder("spark.kubernetes.executor.apiPollingInterval") - .doc("Interval between polls against the Kubernetes API server to inspect the" + - " state of executors.") + .doc("Interval between polls against the Kubernetes API server to inspect the " + + "state of executors.") .timeConf(TimeUnit.MILLISECONDS) .checkValue(interval => interval > 0, s"API server polling interval must be a" + s" positive time value.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala index fde187cdc30d8..267321fabec6b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala @@ -22,8 +22,8 @@ private[spark] trait ExecutorPodsEventQueue { def addSubscriber(processBatchIntervalMillis: Long)(onNextBatch: Seq[Pod] => Unit): Unit - def stopProcessingEvents(): Unit + def stop(): Unit - def pushPodUpdate(updatedPod: Pod): Unit + def enqueue(updatedPod: Pod): Unit } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala index 124f724890bd4..77b8e3901bb2c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala @@ -62,12 +62,12 @@ private[spark] class ExecutorPodsEventQueueImpl( }) } - def stopProcessingEvents(): Unit = { + def stop(): Unit = { observedDisposables.foreach(_.dispose()) eventsObservable.onComplete() } - def pushPodUpdate(updatedPod: Pod): Unit = eventsObservable.onNext(updatedPod) + def enqueue(updatedPod: Pod): Unit = eventsObservable.onNext(updatedPod) private def toReactivexConsumer[T](consumer: T => Unit): Consumer[T] = { new Consumer[T] { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala index 5b36cbdf320c6..12ea449f603e1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala @@ -39,10 +39,7 @@ private[spark] class ExecutorPodsLifecycleEventHandler( // job-breaking if we remove executors more than once but it's ideal if we make an attempt // to avoid doing so. Expire cache entries so that this data structure doesn't grow beyond // bounds. - removedExecutorsCache: Cache[java.lang.Long, java.lang.Long] = - CacheBuilder.newBuilder() - .expireAfterWrite(3, TimeUnit.MINUTES) - .build[java.lang.Long, java.lang.Long]()) extends Logging { + removedExecutorsCache: Cache[java.lang.Long, java.lang.Long]) { import ExecutorPodsLifecycleEventHandler._ diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala index 0da213b9d8ff5..1c8bb337f5850 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala @@ -58,7 +58,7 @@ private[spark] class ExecutorPodsPollingEventSource( .list() .getItems .asScala - .foreach(eventQueue.pushPodUpdate) + .foreach(eventQueue.enqueue) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala index 778574574ad81..459ff78838cc1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala @@ -51,7 +51,7 @@ private[spark] class ExecutorPodsWatchEventSource( private class ExecutorPodsWatcher extends Watcher[Pod] { override def eventReceived(action: Action, pod: Pod): Unit = { - podsEventQueue.pushPodUpdate(pod) + podsEventQueue.enqueue(pod) } override def onClose(e: KubernetesClientException): Unit = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index c44ad092d681d..ee211d24dc76e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -17,7 +17,9 @@ package org.apache.spark.scheduler.cluster.k8s import java.io.File +import java.util.concurrent.TimeUnit +import com.google.common.cache.CacheBuilder import io.fabric8.kubernetes.client.Config import org.apache.spark.{SparkContext, SparkException} @@ -65,11 +67,15 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit .newDaemonCachedThreadPool("kubernetes-executor-pods-event-handlers") val eventQueue = new ExecutorPodsEventQueueImpl( bufferEventsExecutor, executeEventSubscribersExecutor) + val removedExecutorsCache = CacheBuilder.newBuilder() + .expireAfterWrite(3, TimeUnit.MINUTES) + .build[java.lang.Long, java.lang.Long]() val executorPodsLifecycleEventHandler = new ExecutorPodsLifecycleEventHandler( sc.conf, new KubernetesExecutorBuilder(), kubernetesClient, - eventQueue) + eventQueue, + removedExecutorsCache) val executorPodsAllocator = new ExecutorPodsAllocator( sc.conf, new KubernetesExecutorBuilder(), kubernetesClient, eventQueue) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 3dee474fb0fc3..dace933db37aa 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler.cluster.k8s import java.util.concurrent.ExecutorService +import com.google.common.util.concurrent.MoreExecutors import io.fabric8.kubernetes.client.KubernetesClient import scala.concurrent.{ExecutionContext, Future} @@ -70,7 +71,9 @@ private[spark] class KubernetesClusterSchedulerBackend( override def stop(): Unit = { super.stop() - eventQueue.stopProcessingEvents() + Utils.tryLogNonFatalError { + eventQueue.stop() + } Utils.tryLogNonFatalError { watchEvents.stop() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala index dea0a0536f73e..8e2d3b28363c2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala @@ -30,9 +30,9 @@ class DeterministicExecutorPodsEventQueue extends ExecutorPodsEventQueue { subscribers += onNextBatch } - override def stopProcessingEvents(): Unit = {} + override def stop(): Unit = {} - override def pushPodUpdate(updatedPod: Pod): Unit = eventBuffer += updatedPod + override def enqueue(updatedPod: Pod): Unit = eventBuffer += updatedPod def notifySubscribers(): Unit = { subscribers.foreach { _(eventBuffer) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 2c46147236670..e7f2487ca41a7 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -101,12 +101,12 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { podsAllocatorUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) eventQueue.notifySubscribers() for (execId <- 1 until podAllocationSize) { - eventQueue.pushPodUpdate(runningExecutor(execId)) + eventQueue.enqueue(runningExecutor(execId)) } eventQueue.notifySubscribers() verify(podOperations, never()).create( podWithAttachedContainerForId(podAllocationSize + 1)) - eventQueue.pushPodUpdate( + eventQueue.enqueue( runningExecutor(podAllocationSize)) eventQueue.notifySubscribers() verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) @@ -119,10 +119,10 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { podsAllocatorUnderTest.setTotalExpectedExecutors(podAllocationSize) eventQueue.notifySubscribers() for (execId <- 1 until podAllocationSize) { - eventQueue.pushPodUpdate(runningExecutor(execId)) + eventQueue.enqueue(runningExecutor(execId)) } val failedPod = failedExecutorWithoutDeletion(podAllocationSize) - eventQueue.pushPodUpdate(failedPod) + eventQueue.enqueue(failedPod) eventQueue.notifySubscribers() verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala index 9c8de20a58095..af9c0ac9751c2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala @@ -84,7 +84,7 @@ class ExecutorPodsEventQueueSuite extends SparkFunSuite with BeforeAndAfter { } private def pushPodWithIndex(index: Int): Unit = - eventQueueUnderTest.pushPodUpdate(podWithIndex(index)) + eventQueueUnderTest.enqueue(podWithIndex(index)) private def podWithIndex(index: Int): Pod = new PodBuilder() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala index a8df695e9abf2..6bd3a7536ee37 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala @@ -74,7 +74,7 @@ class ExecutorPodsLifecycleEventHandlerSuite extends SparkFunSuite with BeforeAn test("When an executor reaches error states immediately, remove from the scheduler backend.") { val failedPod = failedExecutorWithoutDeletion(1) - eventQueue.pushPodUpdate(failedPod) + eventQueue.enqueue(failedPod) eventQueue.notifySubscribers() val msg = exitReasonMessage(1, failedPod) val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) @@ -84,8 +84,8 @@ class ExecutorPodsLifecycleEventHandlerSuite extends SparkFunSuite with BeforeAn test("Don't remove executors twice from Spark but remove from K8s repeatedly.") { val failedPod = failedExecutorWithoutDeletion(1) - eventQueue.pushPodUpdate(failedPod) - eventQueue.pushPodUpdate(failedPod) + eventQueue.enqueue(failedPod) + eventQueue.enqueue(failedPod) eventQueue.notifySubscribers() val msg = exitReasonMessage(1, failedPod) val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala index 644ce7b018e21..36a945916f4da 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala @@ -79,8 +79,8 @@ class ExecutorPodsPollingEventSourceSuite extends SparkFunSuite with BeforeAndAf runningExecutor(2)) .build()) pollingExecutor.tick(pollingInterval, TimeUnit.MILLISECONDS) - verify(eventQueue).pushPodUpdate(runningExecutor(1)) - verify(eventQueue).pushPodUpdate(runningExecutor(2)) + verify(eventQueue).enqueue(runningExecutor(1)) + verify(eventQueue).enqueue(runningExecutor(2)) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala index 465ab845e09e7..cabcc4be01374 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala @@ -69,7 +69,7 @@ class ExecutorPodsWatchEventSourceSuite extends SparkFunSuite with BeforeAndAfte test("Watch events should be pushed to the queue.") { watch.getValue.eventReceived(Action.ADDED, runningExecutor(1)) watch.getValue.eventReceived(Action.MODIFIED, runningExecutor(2)) - verify(eventQueue).pushPodUpdate(runningExecutor(1)) - verify(eventQueue).pushPodUpdate(runningExecutor(2)) + verify(eventQueue).enqueue(runningExecutor(1)) + verify(eventQueue).enqueue(runningExecutor(2)) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 29c99c80d7b33..95ab91063946e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -115,7 +115,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn when(podOperations.withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)).thenReturn(labeledPods) when(labeledPods.withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)).thenReturn(labeledPods) schedulerBackendUnderTest.stop() - verify(eventQueue).stopProcessingEvents() + verify(eventQueue).stop() verify(watchEvents).stop() verify(pollEvents).stop() verify(labeledPods).delete() From 57ea5ddc408840e4e9629312743ae1e7b0ab40e7 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 25 May 2018 11:17:48 -0700 Subject: [PATCH 26/47] Address more comments --- .../scala/org/apache/spark/util/ThreadUtils.scala | 15 ++++++++++++--- .../cluster/k8s/ExecutorPodsEventQueueImpl.scala | 8 +++++--- .../k8s/ExecutorPodsLifecycleEventHandler.scala | 3 --- .../k8s/ExecutorPodsPollingEventSource.scala | 3 ++- .../k8s/KubernetesClusterSchedulerBackend.scala | 4 ++-- .../cluster/k8s/ExecutorPodsEventQueueSuite.scala | 6 +++--- 6 files changed, 24 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala index 81aaf79db0c13..224b04cff97ba 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala @@ -19,13 +19,12 @@ package org.apache.spark.util import java.util.concurrent._ +import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} import scala.concurrent.{Awaitable, ExecutionContext, ExecutionContextExecutor} -import scala.concurrent.duration.Duration +import scala.concurrent.duration.{Duration, FiniteDuration} import scala.concurrent.forkjoin.{ForkJoinPool => SForkJoinPool, ForkJoinWorkerThread => SForkJoinWorkerThread} import scala.util.control.NonFatal -import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} - import org.apache.spark.SparkException private[spark] object ThreadUtils { @@ -227,4 +226,14 @@ private[spark] object ThreadUtils { } } // scalastyle:on awaitready + + def shutdown( + executor: ExecutorService, + gracePeriod: Duration = FiniteDuration(30, TimeUnit.SECONDS)): Unit = { + executor.shutdown() + executor.awaitTermination(gracePeriod.toMillis, TimeUnit.MILLISECONDS) + if (!executor.isShutdown) { + executor.shutdownNow() + } + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala index 77b8e3901bb2c..30ca66af7805b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.scheduler.cluster.k8s -import java.util.concurrent.{Executor, ScheduledExecutorService, TimeUnit} +import java.util.concurrent.{ExecutorService, ScheduledExecutorService, TimeUnit} import com.google.common.collect.Lists import io.fabric8.kubernetes.api.model.Pod @@ -27,11 +27,11 @@ import io.reactivex.subjects.PublishSubject import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class ExecutorPodsEventQueueImpl( bufferEventsExecutor: ScheduledExecutorService, - executeSubscriptionsExecutor: Executor) + executeSubscriptionsExecutor: ExecutorService) extends ExecutorPodsEventQueue { private val eventsObservable = PublishSubject.create[Pod]() @@ -65,6 +65,8 @@ private[spark] class ExecutorPodsEventQueueImpl( def stop(): Unit = { observedDisposables.foreach(_.dispose()) eventsObservable.onComplete() + ThreadUtils.shutdown(bufferEventsExecutor) + ThreadUtils.shutdown(executeSubscriptionsExecutor) } def enqueue(updatedPod: Pod): Unit = eventsObservable.onNext(updatedPod) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala index 12ea449f603e1..d5ffdbe740e6f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala @@ -16,8 +16,6 @@ */ package org.apache.spark.scheduler.cluster.k8s -import java.util.concurrent.TimeUnit - import com.google.common.cache.{Cache, CacheBuilder} import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.KubernetesClient @@ -26,7 +24,6 @@ import scala.collection.JavaConverters._ import org.apache.spark.SparkConf 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.ExecutorExited import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala index 1c8bb337f5850..e41ec6539bc09 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.util.ThreadUtils private[spark] class ExecutorPodsPollingEventSource( conf: SparkConf, @@ -46,7 +47,7 @@ private[spark] class ExecutorPodsPollingEventSource( pollingFuture.cancel(true) pollingFuture = null } - pollingExecutor.shutdown() + ThreadUtils.shutdown(pollingExecutor) } private class PollRunnable(applicationId: String) extends Runnable { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index dace933db37aa..9dee8a373449e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -26,7 +26,7 @@ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.rpc.{RpcAddress, RpcEnv} import org.apache.spark.scheduler.{ExecutorLossReason, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SchedulerBackendUtils} -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, @@ -91,7 +91,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } Utils.tryLogNonFatalError { - kubernetesClient.close() + ThreadUtils.shutdown(requestExecutorsService) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala index af9c0ac9751c2..fae4c31cdff5d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala @@ -29,12 +29,12 @@ import org.apache.spark.SparkFunSuite class ExecutorPodsEventQueueSuite extends SparkFunSuite with BeforeAndAfter { private var eventBufferScheduler: DeterministicScheduler = _ - private var executeSubscriptionsExecutor: DeterministicExecutor = _ + private var executeSubscriptionsExecutor: DeterministicScheduler = _ private var eventQueueUnderTest: ExecutorPodsEventQueueImpl = _ before { eventBufferScheduler = new DeterministicScheduler() - executeSubscriptionsExecutor = new DeterministicExecutor + executeSubscriptionsExecutor = new DeterministicScheduler() eventQueueUnderTest = new ExecutorPodsEventQueueImpl( eventBufferScheduler, executeSubscriptionsExecutor) @@ -75,7 +75,7 @@ class ExecutorPodsEventQueueSuite extends SparkFunSuite with BeforeAndAfter { val receivedInitialBuffer = new AtomicReference[Seq[Pod]](null) eventQueueUnderTest.addSubscriber(1000) { receivedInitialBuffer.set } assert(receivedInitialBuffer.get == null) - executeSubscriptionsExecutor.runPendingCommands() + executeSubscriptionsExecutor.runUntilIdle() assert(receivedInitialBuffer.get != null) } From b30ed39ebecc72cadfc9ec20b135d60f618762a4 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 29 May 2018 13:26:15 -0700 Subject: [PATCH 27/47] Address comments. --- LICENSE | 1 + licenses/LICENSE-jmock.txt | 28 +++++++++++++++++++ .../org/apache/spark/deploy/k8s/Config.scala | 4 +-- .../cluster/k8s/ExecutorPodsAllocator.scala | 2 +- .../ExecutorPodsLifecycleEventHandler.scala | 1 + 5 files changed, 33 insertions(+), 3 deletions(-) create mode 100644 licenses/LICENSE-jmock.txt diff --git a/LICENSE b/LICENSE index 820f14dbdeed0..cc1f580207a75 100644 --- a/LICENSE +++ b/LICENSE @@ -237,6 +237,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (BSD 3 Clause) netlib core (com.github.fommil.netlib:core:1.1.2 - https://github.com/fommil/netlib-java/core) (BSD 3 Clause) JPMML-Model (org.jpmml:pmml-model:1.2.7 - https://github.com/jpmml/jpmml-model) + (BSD 3 Clause) jmock (org.jmock:jmock-junit4:2.8.4 - http://jmock.org/) (BSD License) AntLR Parser Generator (antlr:antlr:2.7.7 - http://www.antlr.org/) (BSD License) ANTLR 4.5.2-1 (org.antlr:antlr4:4.5.2-1 - http://wwww.antlr.org/) (BSD licence) ANTLR ST4 4.0.4 (org.antlr:ST4:4.0.4 - http://www.stringtemplate.org) diff --git a/licenses/LICENSE-jmock.txt b/licenses/LICENSE-jmock.txt new file mode 100644 index 0000000000000..ed7964fe3d9ef --- /dev/null +++ b/licenses/LICENSE-jmock.txt @@ -0,0 +1,28 @@ +Copyright (c) 2000-2017, jMock.org +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +Redistributions of source code must retain the above copyright notice, +this list of conditions and the following disclaimer. Redistributions +in binary form must reproduce the above copyright notice, this list of +conditions and the following disclaimer in the documentation and/or +other materials provided with the distribution. + +Neither the name of jMock nor the names of its contributors may be +used to endorse or promote products derived from this software without +specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 351ec30e6b68c..a5353e0b62130 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -160,7 +160,7 @@ private[spark] object Config extends Logging { "state of executors.") .timeConf(TimeUnit.MILLISECONDS) .checkValue(interval => interval > 0, s"API server polling interval must be a" + - s" positive time value.") + " positive time value.") .createWithDefaultString("30s") val KUBERNETES_EXECUTOR_EVENT_PROCESSING_INTERVAL = @@ -169,7 +169,7 @@ private[spark] object Config extends Logging { " Kubernetes API.") .timeConf(TimeUnit.MILLISECONDS) .checkValue(interval => interval > 0, s"Event processing interval must be a positive" + - s" time value.") + " time value.") .createWithDefaultString("1s") val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index cf97491959709..241af73d0ca5b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -114,7 +114,7 @@ private[spark] class ExecutorPodsAllocator( " executors. Not scaling up further.") } else if (pendingExecutors.nonEmpty) { logInfo(s"Still waiting for ${pendingExecutors.size} executors to begin running before" + - s" requesting for more executors.") + " requesting for more executors.") } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala index d5ffdbe740e6f..37c4995a646c9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala @@ -70,6 +70,7 @@ private[spark] class ExecutorPodsLifecycleEventHandler( .delete() } removeExecutorFromSpark(schedulerBackend, updatedPod, execId) + case _ => } } } From 5b9c00fa39d1c83435ca65de5394345e5d6f1f00 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 29 May 2018 17:08:33 -0700 Subject: [PATCH 28/47] Close k8s client --- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 9dee8a373449e..b0adf9e87fc8e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -93,6 +93,10 @@ private[spark] class KubernetesClusterSchedulerBackend( Utils.tryLogNonFatalError { ThreadUtils.shutdown(requestExecutorsService) } + + Utils.tryLogNonFatalError { + kubernetesClient.close() + } } override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { From 260d82ca9fbbd16ad8174d0dafa2f95bc177a219 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 30 May 2018 16:57:15 -0700 Subject: [PATCH 29/47] Addressed comments. --- .../cluster/k8s/ExecutorPodsAllocator.scala | 8 +++--- .../ExecutorPodsLifecycleEventHandler.scala | 25 +++++++++++-------- 2 files changed, 19 insertions(+), 14 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 241af73d0ca5b..10dd3dbe27ed3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -65,13 +65,13 @@ private[spark] class ExecutorPodsAllocator( def start(applicationId: String): Unit = { eventQueue.addSubscriber(podAllocationDelay) { updatedPods => - processUpdatedPodEvents(applicationId, updatedPods) + processUpdatedPods(applicationId, updatedPods) } } def setTotalExpectedExecutors(total: Int): Unit = totalExpectedExecutors.set(total) - private def processUpdatedPodEvents(applicationId: String, updatedPods: Seq[Pod]): Unit = { + private def processUpdatedPods(applicationId: String, updatedPods: Seq[Pod]): Unit = { updatedPods.foreach { updatedPod => val execId = updatedPod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL).toLong val phase = updatedPod.getStatus.getPhase.toLowerCase @@ -109,11 +109,11 @@ private[spark] class ExecutorPodsAllocator( kubernetesClient.pods().create(podWithAttachedContainer) pendingExecutors += newExecutorId } - } else if (currentRunningExecutors == currentTotalExpectedExecutors) { + } else if (currentRunningExecutors >= currentTotalExpectedExecutors) { logDebug("Current number of running executors is equal to the number of requested" + " executors. Not scaling up further.") } else if (pendingExecutors.nonEmpty) { - logInfo(s"Still waiting for ${pendingExecutors.size} executors to begin running before" + + logDebug(s"Still waiting for ${pendingExecutors.size} executors to begin running before" + " requesting for more executors.") } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala index 37c4995a646c9..76cf58697e768 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala @@ -59,22 +59,27 @@ private[spark] class ExecutorPodsLifecycleEventHandler( updatedPod.getStatus.getPhase.toLowerCase match { // TODO (SPARK-24135) - handle more classes of errors case "error" | "failed" | "succeeded" => - // If deletion failed on a previous try, we can try again if resync informs us the pod - // is still around. - // Delete as best attempt - duplicate deletes will throw an exception but the end state - // of getting rid of the pod is what matters. - Utils.tryLogNonFatalError { - kubernetesClient - .pods() - .withName(updatedPod.getMetadata.getName) - .delete() - } + removeExecutorFromK8s(schedulerBackend, updatedPod, execId) removeExecutorFromSpark(schedulerBackend, updatedPod, execId) case _ => } } } + private def removeExecutorFromK8s( + schedulerBackend: KubernetesClusterSchedulerBackend, updatedPod: Pod, execId: Long): Unit = { + // If deletion failed on a previous try, we can try again if resync informs us the pod + // is still around. + // Delete as best attempt - duplicate deletes will throw an exception but the end state + // of getting rid of the pod is what matters. + Utils.tryLogNonFatalError { + kubernetesClient + .pods() + .withName(updatedPod.getMetadata.getName) + .delete() + } + } + private def removeExecutorFromSpark( schedulerBackend: KubernetesClusterSchedulerBackend, updatedPod: Pod, From bd03451a13d1d44fea3a365aad49843a9f24fc91 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 1 Jun 2018 12:56:05 -0700 Subject: [PATCH 30/47] Addressed comments. Factor out state determination to a separate class. Allow clients to specify state handlers as partial functions. --- .../k8s/ExecutorPodBatchSubscriber.scala | 53 +++++++++++++ .../cluster/k8s/ExecutorPodStates.scala | 41 ++++++++++ .../cluster/k8s/ExecutorPodsAllocator.scala | 31 ++++---- .../cluster/k8s/ExecutorPodsEventQueue.scala | 4 +- .../k8s/ExecutorPodsEventQueueImpl.scala | 6 +- .../ExecutorPodsLifecycleEventHandler.scala | 38 ++++----- .../KubernetesClusterSchedulerBackend.scala | 1 - .../DeterministicExecutorPodsEventQueue.scala | 14 ++-- .../k8s/ExecutorLifecycleTestUtils.scala | 32 +++++++- .../k8s/ExecutorPodBatchSubscriberSuite.scala | 79 +++++++++++++++++++ .../k8s/ExecutorPodsAllocatorSuite.scala | 6 +- .../k8s/ExecutorPodsEventQueueSuite.scala | 39 +++++++-- 12 files changed, 282 insertions(+), 62 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriber.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriberSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriber.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriber.scala new file mode 100644 index 0000000000000..d1e8523279d65 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriber.scala @@ -0,0 +1,53 @@ +/* + * 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 io.fabric8.kubernetes.api.model.Pod + +import org.apache.spark.internal.Logging + +private[spark] class ExecutorPodBatchSubscriber( + updatedPodSubscriber: PartialFunction[ExecutorPodState, Unit], + processedBatchSubscriber: () => Unit) + extends Logging { + + def onNextBatch(pods: Iterable[Pod]): Unit = { + pods.foreach { pod => + val state = if (isDeleted(pod)) { + PodDeleted(pod) + } else { + val phase = pod.getStatus.getPhase.toLowerCase + phase match { + case "running" => + PodRunning(pod) + case "failed" => + PodFailed(pod) + case "succeeded" => + PodSucceeded(pod) + case _ => + logWarning(s"Received unknown phase $phase for executor pod with name" + + s" ${pod.getMetadata.getName} in namespace ${pod.getMetadata.getNamespace}") + PodUnknown(pod) + } + } + updatedPodSubscriber(state) + } + processedBatchSubscriber() + } + + private def isDeleted(pod: Pod): Boolean = pod.getMetadata.getDeletionTimestamp != null +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala new file mode 100644 index 0000000000000..2b0327deab9a2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala @@ -0,0 +1,41 @@ +/* + * 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 io.fabric8.kubernetes.api.model.Pod + +import org.apache.spark.deploy.k8s.Constants._ + +sealed trait ExecutorPodState { + def pod: Pod + + final def execId(): Long = pod + .getMetadata + .getLabels + .get(SPARK_EXECUTOR_ID_LABEL) + .toLong +} + +case class PodRunning(pod: Pod) extends ExecutorPodState + +case class PodSucceeded(pod: Pod) extends ExecutorPodState + +case class PodFailed(pod: Pod) extends ExecutorPodState + +case class PodDeleted(pod: Pod) extends ExecutorPodState + +case class PodUnknown(pod: Pod) extends ExecutorPodState diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 10dd3dbe27ed3..e8dc144487b91 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -64,27 +64,26 @@ private[spark] class ExecutorPodsAllocator( private val runningExecutors = mutable.Set.empty[Long] def start(applicationId: String): Unit = { - eventQueue.addSubscriber(podAllocationDelay) { updatedPods => - processUpdatedPods(applicationId, updatedPods) - } + eventQueue.addSubscriber( + podAllocationDelay, + new ExecutorPodBatchSubscriber( + processUpdatedPod(applicationId), + () => postProcessBatch(applicationId))) } def setTotalExpectedExecutors(total: Int): Unit = totalExpectedExecutors.set(total) - private def processUpdatedPods(applicationId: String, updatedPods: Seq[Pod]): Unit = { - updatedPods.foreach { updatedPod => - val execId = updatedPod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL).toLong - val phase = updatedPod.getStatus.getPhase.toLowerCase - phase match { - case "running" => - pendingExecutors -= execId - runningExecutors += execId - case "failed" | "succeeded" | "error" => - pendingExecutors -= execId - runningExecutors -= execId - } - } + private def processUpdatedPod(applicationId: String): PartialFunction[ExecutorPodState, Unit] = { + case running @ PodRunning(_) => + pendingExecutors -= running.execId() + runningExecutors += running.execId() + case completed @ (PodSucceeded(_) | PodDeleted(_) | PodFailed(_)) => + pendingExecutors -= completed.execId() + runningExecutors -= completed.execId() + case _ => + } + private def postProcessBatch(applicationId: String): Unit = { val currentRunningExecutors = runningExecutors.size val currentTotalExpectedExecutors = totalExpectedExecutors.get if (pendingExecutors.isEmpty && currentRunningExecutors < currentTotalExpectedExecutors) { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala index 267321fabec6b..4b9abbfaf1ae1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala @@ -20,7 +20,9 @@ import io.fabric8.kubernetes.api.model.Pod private[spark] trait ExecutorPodsEventQueue { - def addSubscriber(processBatchIntervalMillis: Long)(onNextBatch: Seq[Pod] => Unit): Unit + def addSubscriber + (processBatchIntervalMillis: Long, + subscriber: ExecutorPodBatchSubscriber): Unit def stop(): Unit diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala index 30ca66af7805b..d06082630a0a5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala @@ -37,7 +37,9 @@ private[spark] class ExecutorPodsEventQueueImpl( private val eventsObservable = PublishSubject.create[Pod]() private val observedDisposables = mutable.Buffer.empty[Disposable] - def addSubscriber(processBatchIntervalMillis: Long)(onNextBatch: Seq[Pod] => Unit): Unit = { + def addSubscriber( + processBatchIntervalMillis: Long, + subscriber: ExecutorPodBatchSubscriber): Unit = { observedDisposables += eventsObservable // Group events in the time window given by the caller. These buffers are then sent // to the caller's lambda at the given interval, with the pod updates that occurred @@ -57,7 +59,7 @@ private[spark] class ExecutorPodsEventQueueImpl( .observeOn(Schedulers.from(executeSubscriptionsExecutor)) .subscribe(toReactivexConsumer { (pods: java.util.List[Pod]) => Utils.tryLogNonFatalError { - onNextBatch(pods.asScala) + subscriber.onNextBatch(pods.asScala) } }) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala index 76cf58697e768..fd79ea4f309b2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala @@ -43,31 +43,25 @@ private[spark] class ExecutorPodsLifecycleEventHandler( private val eventProcessingInterval = conf.get(KUBERNETES_EXECUTOR_EVENT_PROCESSING_INTERVAL) def start(schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { - podsEventQueue.addSubscriber(eventProcessingInterval) { updatedPods => - updatedPods.foreach { updatedPod => - processUpdatedPod(schedulerBackend, updatedPod) - } - } + podsEventQueue.addSubscriber( + eventProcessingInterval, + new ExecutorPodBatchSubscriber( + processUpdatedPod(schedulerBackend), + () => {})) } private def processUpdatedPod( - schedulerBackend: KubernetesClusterSchedulerBackend, updatedPod: Pod) = { - val execId = updatedPod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL).toLong - if (isDeleted(updatedPod)) { - removeExecutorFromSpark(schedulerBackend, updatedPod, execId) - } else { - updatedPod.getStatus.getPhase.toLowerCase match { - // TODO (SPARK-24135) - handle more classes of errors - case "error" | "failed" | "succeeded" => - removeExecutorFromK8s(schedulerBackend, updatedPod, execId) - removeExecutorFromSpark(schedulerBackend, updatedPod, execId) - case _ => - } - } + schedulerBackend: KubernetesClusterSchedulerBackend) + : PartialFunction[ExecutorPodState, Unit] = { + case deleted @ PodDeleted(pod) => + removeExecutorFromSpark(schedulerBackend, pod, deleted.execId()) + case errorOrSucceeded @ (PodFailed(_) | PodSucceeded(_)) => + removeExecutorFromK8s(errorOrSucceeded.pod) + removeExecutorFromSpark(schedulerBackend, errorOrSucceeded.pod, errorOrSucceeded.execId()) + case _ => } - private def removeExecutorFromK8s( - schedulerBackend: KubernetesClusterSchedulerBackend, updatedPod: Pod, execId: Long): Unit = { + private def removeExecutorFromK8s(updatedPod: Pod): Unit = { // If deletion failed on a previous try, we can try again if resync informs us the pod // is still around. // Delete as best attempt - duplicate deletes will throw an exception but the end state @@ -82,11 +76,11 @@ private[spark] class ExecutorPodsLifecycleEventHandler( private def removeExecutorFromSpark( schedulerBackend: KubernetesClusterSchedulerBackend, - updatedPod: Pod, + pod: Pod, execId: Long): Unit = { if (removedExecutorsCache.getIfPresent(execId) == null) { removedExecutorsCache.put(execId, execId) - val exitReason = findExitReason(updatedPod, execId) + val exitReason = findExitReason(pod, execId) schedulerBackend.doRemoveExecutor(execId.toString, exitReason) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index b0adf9e87fc8e..5f00fda396297 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler.cluster.k8s import java.util.concurrent.ExecutorService -import com.google.common.util.concurrent.MoreExecutors import io.fabric8.kubernetes.client.KubernetesClient import scala.concurrent.{ExecutionContext, Future} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala index 8e2d3b28363c2..f2912e4efec28 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala @@ -22,12 +22,12 @@ import scala.collection.mutable class DeterministicExecutorPodsEventQueue extends ExecutorPodsEventQueue { private val eventBuffer = mutable.Buffer.empty[Pod] - private val subscribers = mutable.Buffer.empty[(Seq[Pod]) => Unit] + private val subscribers = mutable.Buffer.empty[ExecutorPodBatchSubscriber] - override def addSubscriber - (processBatchIntervalMillis: Long) - (onNextBatch: (Seq[Pod]) => Unit): Unit = { - subscribers += onNextBatch + override def addSubscriber( + processBatchIntervalMillis: Long, + subscriber: ExecutorPodBatchSubscriber): Unit = { + subscribers += subscriber } override def stop(): Unit = {} @@ -35,7 +35,9 @@ class DeterministicExecutorPodsEventQueue extends ExecutorPodsEventQueue { override def enqueue(updatedPod: Pod): Unit = eventBuffer += updatedPod def notifySubscribers(): Unit = { - subscribers.foreach { _(eventBuffer) } + subscribers.foreach { subscriber => + subscriber.onNextBatch(eventBuffer) + } eventBuffer.clear() } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala index 50312f6ff7186..4d255a5f3bebd 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala @@ -28,7 +28,7 @@ object ExecutorLifecycleTestUtils { def failedExecutorWithoutDeletion(executorId: Int): Pod = { new PodBuilder(podWithAttachedContainerForId(executorId)) .editOrNewStatus() - .withPhase("error") + .withPhase("failed") .addNewContainerStatus() .withName("spark-executor") .withImage("k8s-spark") @@ -63,8 +63,32 @@ object ExecutorLifecycleTestUtils { .build() } + def succeededExecutor(executorId: Int): Pod = { + new PodBuilder(podWithAttachedContainerForId(executorId)) + .editOrNewStatus() + .withPhase("succeeded") + .endStatus() + .build() + } + + def deletedExecutor(executorId: Int): Pod = { + new PodBuilder(podWithAttachedContainerForId(executorId)) + .editOrNewMetadata() + .withNewDeletionTimestamp("523012521") + .endMetadata() + .build() + } + + def unknownExecutor(executorId: Int): Pod = { + new PodBuilder(podWithAttachedContainerForId(executorId)) + .editOrNewStatus() + .withPhase("unknown") + .endStatus() + .build() + } + def podWithAttachedContainerForId(executorId: Int): Pod = { - val sparkPod = executorPodWithId(executorId.toString) + val sparkPod = executorPodWithId(executorId) val podWithAttachedContainer = new PodBuilder(sparkPod.pod) .editOrNewSpec() .addToContainers(sparkPod.container) @@ -73,13 +97,13 @@ object ExecutorLifecycleTestUtils { podWithAttachedContainer } - def executorPodWithId(executorId: String): SparkPod = { + def executorPodWithId(executorId: Int): SparkPod = { val pod = new PodBuilder() .withNewMetadata() .withName(s"spark-executor-$executorId") .addToLabels(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID) .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) - .addToLabels(SPARK_EXECUTOR_ID_LABEL, executorId) + .addToLabels(SPARK_EXECUTOR_ID_LABEL, executorId.toString) .endMetadata() .build() val container = new ContainerBuilder() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriberSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriberSuite.scala new file mode 100644 index 0000000000000..6f6bcc04c0866 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriberSuite.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler.cluster.k8s + +import java.util.concurrent.atomic.AtomicInteger + +import io.fabric8.kubernetes.api.model.Pod + +import org.apache.spark.SparkFunSuite + +private[spark] class ExecutorPodBatchSubscriberSuite extends SparkFunSuite { + + test("Check the correctness of states.") { + val statesDetectionFunction = new StatesDetectionFunction() + val subscriber = new ExecutorPodBatchSubscriber(statesDetectionFunction, () => {}) + val running = ExecutorLifecycleTestUtils.runningExecutor(1) + val failed = ExecutorLifecycleTestUtils.failedExecutorWithoutDeletion(2) + val deleted = ExecutorLifecycleTestUtils.deletedExecutor(3) + val succeeded = ExecutorLifecycleTestUtils.succeededExecutor(4) + val unknown = ExecutorLifecycleTestUtils.unknownExecutor(5) + subscriber.onNextBatch(Seq(running, failed, deleted, succeeded, unknown)) + assert(statesDetectionFunction.podRunning === PodRunning(running)) + assert(statesDetectionFunction.podFailed === PodFailed(failed)) + assert(statesDetectionFunction.podDeleted === PodDeleted(deleted)) + assert(statesDetectionFunction.podSucceeded === PodSucceeded(succeeded)) + assert(statesDetectionFunction.podUnknown === PodUnknown(unknown)) + } + + test("Invoke post batch processor after every batch.") { + val batchProcessedCounter = new AtomicInteger(0) + val subscriber = new ExecutorPodBatchSubscriber( + new StatesDetectionFunction(), () => batchProcessedCounter.getAndIncrement()) + subscriber.onNextBatch(Seq.empty[Pod]) + assert(batchProcessedCounter.get === 1) + subscriber.onNextBatch(Seq.empty[Pod]) + assert(batchProcessedCounter.get === 2) + } + + private class StatesDetectionFunction extends PartialFunction[ExecutorPodState, Unit] { + var podRunning: PodRunning = _ + var podFailed: PodFailed = _ + var podDeleted: PodDeleted = _ + var podSucceeded: PodSucceeded = _ + var podUnknown: PodUnknown = _ + + override def isDefinedAt(state: ExecutorPodState): Boolean = { + state match { + case PodRunning(_) | PodFailed(_) | PodDeleted(_) | PodSucceeded(_) | PodUnknown(_) => true + case _ => false + } + } + + override def apply(state: ExecutorPodState): Unit = { + state match { + case running @ PodRunning(_) => podRunning = running + case failed @ PodFailed(_) => podFailed = failed + case deleted @ PodDeleted(_) => podDeleted = deleted + case succeeded @ PodSucceeded(_) => podSucceeded = succeeded + case unknown @ PodUnknown(_) => podUnknown = unknown + case other @ _ => throw new IllegalArgumentException(s"Unknown state $other") + } + } + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index e7f2487ca41a7..fa46446b76e89 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -51,10 +51,6 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) - private val podAllocationDelay = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) - - private var namedExecutorPods: mutable.Map[String, PodResource[Pod, DoneablePod]] = _ - @Mock private var kubernetesClient: KubernetesClient = _ @@ -132,7 +128,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { override def answer(invocation: InvocationOnMock): SparkPod = { val k8sConf = invocation.getArgumentAt( 0, classOf[KubernetesConf[KubernetesExecutorSpecificConf]]) - executorPodWithId(k8sConf.roleSpecificConf.executorId) + executorPodWithId(k8sConf.roleSpecificConf.executorId.toInt) } } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala index fae4c31cdff5d..ecdeae65e1067 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala @@ -20,7 +20,11 @@ import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicReference import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} -import org.jmock.lib.concurrent.{DeterministicExecutor, DeterministicScheduler} +import org.jmock.lib.concurrent.DeterministicScheduler +import org.mockito.Matchers.any +import org.mockito.Mockito.{mock, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter import scala.collection.mutable @@ -43,8 +47,8 @@ class ExecutorPodsEventQueueSuite extends SparkFunSuite with BeforeAndAfter { test("Subscribers get notified of events periodically.") { val receivedEvents1 = mutable.Buffer.empty[Pod] val receivedEvents2 = mutable.Buffer.empty[Pod] - eventQueueUnderTest.addSubscriber(1000) { receivedEvents1.appendAll(_) } - eventQueueUnderTest.addSubscriber(2000) { receivedEvents2.appendAll(_) } + eventQueueUnderTest.addSubscriber(1000, testBatchSubscriber(receivedEvents1)) + eventQueueUnderTest.addSubscriber(2000, testBatchSubscriber(receivedEvents2)) pushPodWithIndex(1) assert(receivedEvents1.isEmpty) @@ -72,8 +76,8 @@ class ExecutorPodsEventQueueSuite extends SparkFunSuite with BeforeAndAfter { } test("Even without sending events, initially receive an empty buffer.") { - val receivedInitialBuffer = new AtomicReference[Seq[Pod]](null) - eventQueueUnderTest.addSubscriber(1000) { receivedInitialBuffer.set } + val receivedInitialBuffer = new AtomicReference[Iterable[Pod]](null) + eventQueueUnderTest.addSubscriber(1000, testSetBufferSubscriber(receivedInitialBuffer)) assert(receivedInitialBuffer.get == null) executeSubscriptionsExecutor.runUntilIdle() assert(receivedInitialBuffer.get != null) @@ -92,4 +96,29 @@ class ExecutorPodsEventQueueSuite extends SparkFunSuite with BeforeAndAfter { .withName(s"pod-$index") .endMetadata() .build() + + private def testBatchSubscriber(eventBuffer: mutable.Buffer[Pod]): ExecutorPodBatchSubscriber = { + val subscriber = mock(classOf[ExecutorPodBatchSubscriber]) + when(subscriber.onNextBatch(any(classOf[Iterable[Pod]]))) + .thenAnswer(new Answer[Unit] { + override def answer(invocationOnMock: InvocationOnMock): Unit = { + val pods = invocationOnMock.getArgumentAt(0, classOf[Iterable[Pod]]) + eventBuffer ++= pods + } + }) + subscriber + } + + private def testSetBufferSubscriber( + eventBuffer: AtomicReference[Iterable[Pod]]): ExecutorPodBatchSubscriber = { + val subscriber = mock(classOf[ExecutorPodBatchSubscriber]) + when(subscriber.onNextBatch(any(classOf[Iterable[Pod]]))) + .thenAnswer(new Answer[Unit] { + override def answer(invocationOnMock: InvocationOnMock): Unit = { + val pods = invocationOnMock.getArgumentAt(0, classOf[Iterable[Pod]]) + eventBuffer.set(pods) + } + }) + subscriber + } } From f294dcaca673320bfca3ffc462b4f13a5d0e6b15 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 4 Jun 2018 13:39:02 -0700 Subject: [PATCH 31/47] Small style --- .../spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala index 4b9abbfaf1ae1..da71678856f42 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala @@ -20,9 +20,7 @@ import io.fabric8.kubernetes.api.model.Pod private[spark] trait ExecutorPodsEventQueue { - def addSubscriber - (processBatchIntervalMillis: Long, - subscriber: ExecutorPodBatchSubscriber): Unit + def addSubscriber(processBatchIntervalMillis: Long, subscriber: ExecutorPodBatchSubscriber): Unit def stop(): Unit From 7bf49ba9860c2ad7bab916aafd331a404c6987b5 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 4 Jun 2018 13:41:14 -0700 Subject: [PATCH 32/47] More small style tweaks --- .../cluster/k8s/ExecutorLifecycleTestUtils.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala index 4d255a5f3bebd..df33015b4cd73 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala @@ -25,7 +25,7 @@ object ExecutorLifecycleTestUtils { val TEST_SPARK_APP_ID = "spark-app-id" - def failedExecutorWithoutDeletion(executorId: Int): Pod = { + def failedExecutorWithoutDeletion(executorId: Long): Pod = { new PodBuilder(podWithAttachedContainerForId(executorId)) .editOrNewStatus() .withPhase("failed") @@ -55,7 +55,7 @@ object ExecutorLifecycleTestUtils { .build() } - def runningExecutor(executorId: Int): Pod = { + def runningExecutor(executorId: Long): Pod = { new PodBuilder(podWithAttachedContainerForId(executorId)) .editOrNewStatus() .withPhase("running") @@ -63,7 +63,7 @@ object ExecutorLifecycleTestUtils { .build() } - def succeededExecutor(executorId: Int): Pod = { + def succeededExecutor(executorId: Long): Pod = { new PodBuilder(podWithAttachedContainerForId(executorId)) .editOrNewStatus() .withPhase("succeeded") @@ -71,7 +71,7 @@ object ExecutorLifecycleTestUtils { .build() } - def deletedExecutor(executorId: Int): Pod = { + def deletedExecutor(executorId: Long): Pod = { new PodBuilder(podWithAttachedContainerForId(executorId)) .editOrNewMetadata() .withNewDeletionTimestamp("523012521") @@ -79,7 +79,7 @@ object ExecutorLifecycleTestUtils { .build() } - def unknownExecutor(executorId: Int): Pod = { + def unknownExecutor(executorId: Long): Pod = { new PodBuilder(podWithAttachedContainerForId(executorId)) .editOrNewStatus() .withPhase("unknown") @@ -87,7 +87,7 @@ object ExecutorLifecycleTestUtils { .build() } - def podWithAttachedContainerForId(executorId: Int): Pod = { + def podWithAttachedContainerForId(executorId: Long): Pod = { val sparkPod = executorPodWithId(executorId) val podWithAttachedContainer = new PodBuilder(sparkPod.pod) .editOrNewSpec() @@ -97,7 +97,7 @@ object ExecutorLifecycleTestUtils { podWithAttachedContainer } - def executorPodWithId(executorId: Int): SparkPod = { + def executorPodWithId(executorId: Long): SparkPod = { val pod = new PodBuilder() .withNewMetadata() .withName(s"spark-executor-$executorId") From b5c0fbf19a1e393a6df18b362558cb39afa2085d Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 4 Jun 2018 13:47:13 -0700 Subject: [PATCH 33/47] Remove unnecessary parens --- .../scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala index d06082630a0a5..4691082e6d5a1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala @@ -57,7 +57,7 @@ private[spark] class ExecutorPodsEventQueueImpl( // Force all triggered events - both the initial event above and the buffered ones in // the following time windows - to execute asynchronously to this call's thread. .observeOn(Schedulers.from(executeSubscriptionsExecutor)) - .subscribe(toReactivexConsumer { (pods: java.util.List[Pod]) => + .subscribe(toReactivexConsumer { pods: java.util.List[Pod] => Utils.tryLogNonFatalError { subscriber.onNextBatch(pods.asScala) } From c4b87d81a520cec74cd1a39a79a3b31a708b95c9 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 4 Jun 2018 13:52:59 -0700 Subject: [PATCH 34/47] Various style fixes --- .../scheduler/cluster/k8s/ExecutorPodsAllocator.scala | 1 - .../scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala | 3 +-- .../cluster/k8s/ExecutorPodsLifecycleEventHandler.scala | 3 +-- .../cluster/k8s/ExecutorPodsWatchEventSource.scala | 3 +-- .../scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala | 5 ++--- .../k8s/ExecutorPodsLifecycleEventHandlerSuite.scala | 7 ++----- .../k8s/KubernetesClusterSchedulerBackendSuite.scala | 4 +--- 7 files changed, 8 insertions(+), 18 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index e8dc144487b91..271ffb653e90b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -24,7 +24,6 @@ import scala.collection.mutable import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesConf import org.apache.spark.internal.Logging diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala index 4691082e6d5a1..598c727238beb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala @@ -30,8 +30,7 @@ import scala.collection.mutable import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class ExecutorPodsEventQueueImpl( - bufferEventsExecutor: ScheduledExecutorService, - executeSubscriptionsExecutor: ExecutorService) + bufferEventsExecutor: ScheduledExecutorService, executeSubscriptionsExecutor: ExecutorService) extends ExecutorPodsEventQueue { private val eventsObservable = PublishSubject.create[Pod]() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala index fd79ea4f309b2..e5a4b10e9422a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala @@ -16,14 +16,13 @@ */ package org.apache.spark.scheduler.cluster.k8s -import com.google.common.cache.{Cache, CacheBuilder} +import com.google.common.cache.Cache import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.JavaConverters._ import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.scheduler.ExecutorExited import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala index 459ff78838cc1..f307375b01fda 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala @@ -27,8 +27,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.util.Utils private[spark] class ExecutorPodsWatchEventSource( - podsEventQueue: ExecutorPodsEventQueue, - kubernetesClient: KubernetesClient) extends Logging { + podsEventQueue: ExecutorPodsEventQueue, kubernetesClient: KubernetesClient) extends Logging { private var watchConnection: Closeable = _ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index fa46446b76e89..33a7246aca178 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -16,16 +16,15 @@ */ package org.apache.spark.scheduler.cluster.k8s -import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder, PodList} +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder} import io.fabric8.kubernetes.client.KubernetesClient -import io.fabric8.kubernetes.client.dsl.{MixedOperation, PodResource} +import io.fabric8.kubernetes.client.dsl.PodResource import org.mockito.{ArgumentMatcher, Matchers, Mock, MockitoAnnotations} import org.mockito.Matchers.any import org.mockito.Mockito.{never, times, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter -import scala.collection.mutable import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala index 6bd3a7536ee37..0273ab90241fa 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.scheduler.cluster.k8s import com.google.common.cache.CacheBuilder -import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodList} +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod} import io.fabric8.kubernetes.client.KubernetesClient -import io.fabric8.kubernetes.client.dsl.{MixedOperation, PodResource} +import io.fabric8.kubernetes.client.dsl.PodResource import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Matchers.any import org.mockito.Mockito.{mock, times, verify, when} @@ -44,9 +44,6 @@ class ExecutorPodsLifecycleEventHandlerSuite extends SparkFunSuite with BeforeAn @Mock private var podOperations: PODS = _ - @Mock - private var driverPodOperations: SINGLE_POD = _ - @Mock private var executorBuilder: KubernetesExecutorBuilder = _ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 95ab91063946e..8f4db544d43ee 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -16,9 +16,7 @@ */ package org.apache.spark.scheduler.cluster.k8s -import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodList} -import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher} -import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, PodResource} +import io.fabric8.kubernetes.client.KubernetesClient import org.jmock.lib.concurrent.DeterministicScheduler import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} import org.mockito.Matchers.{eq => mockitoEq} From 8615c067328c0c64d0d048922b221477580acdb4 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 7 Jun 2018 18:15:38 -0700 Subject: [PATCH 35/47] Process cluster snapshots instead of deltas. --- .../k8s/ExecutorPodBatchSubscriber.scala | 53 ------- .../cluster/k8s/ExecutorPodStates.scala | 10 +- .../cluster/k8s/ExecutorPodsAllocator.scala | 94 ++++++----- ...ala => ExecutorPodsLifecycleManager.scala} | 59 ++++--- .../k8s/ExecutorPodsPollingEventSource.scala | 7 +- .../cluster/k8s/ExecutorPodsSnapshot.scala | 74 +++++++++ ...scala => ExecutorPodsSnapshotsStore.scala} | 9 +- ...a => ExecutorPodsSnapshotsStoreImpl.scala} | 47 ++++-- .../k8s/ExecutorPodsWatchEventSource.scala | 5 +- .../k8s/KubernetesClusterManager.scala | 26 ++-- .../KubernetesClusterSchedulerBackend.scala | 10 +- ...rministicExecutorPodsSnapshotsStore.scala} | 29 ++-- .../k8s/ExecutorLifecycleTestUtils.scala | 8 + .../k8s/ExecutorPodBatchSubscriberSuite.scala | 79 ---------- .../k8s/ExecutorPodsAllocatorSuite.scala | 64 +++++--- .../k8s/ExecutorPodsEventQueueSuite.scala | 124 --------------- ...> ExecutorPodsLifecycleManagerSuite.scala} | 35 +++-- .../ExecutorPodsPollingEventSourceSuite.scala | 5 +- .../k8s/ExecutorPodsSnapshotSuite.scala | 60 +++++++ .../k8s/ExecutorPodsSnapshotsStoreSuite.scala | 146 ++++++++++++++++++ .../ExecutorPodsWatchEventSourceSuite.scala | 6 +- ...bernetesClusterSchedulerBackendSuite.scala | 4 +- 22 files changed, 546 insertions(+), 408 deletions(-) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriber.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/{ExecutorPodsLifecycleEventHandler.scala => ExecutorPodsLifecycleManager.scala} (67%) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/{ExecutorPodsEventQueue.scala => ExecutorPodsSnapshotsStore.scala} (78%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/{ExecutorPodsEventQueueImpl.scala => ExecutorPodsSnapshotsStoreImpl.scala} (65%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/{DeterministicExecutorPodsEventQueue.scala => DeterministicExecutorPodsSnapshotsStore.scala} (57%) delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriberSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/{ExecutorPodsLifecycleEventHandlerSuite.scala => ExecutorPodsLifecycleManagerSuite.scala} (76%) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriber.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriber.scala deleted file mode 100644 index d1e8523279d65..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriber.scala +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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 io.fabric8.kubernetes.api.model.Pod - -import org.apache.spark.internal.Logging - -private[spark] class ExecutorPodBatchSubscriber( - updatedPodSubscriber: PartialFunction[ExecutorPodState, Unit], - processedBatchSubscriber: () => Unit) - extends Logging { - - def onNextBatch(pods: Iterable[Pod]): Unit = { - pods.foreach { pod => - val state = if (isDeleted(pod)) { - PodDeleted(pod) - } else { - val phase = pod.getStatus.getPhase.toLowerCase - phase match { - case "running" => - PodRunning(pod) - case "failed" => - PodFailed(pod) - case "succeeded" => - PodSucceeded(pod) - case _ => - logWarning(s"Received unknown phase $phase for executor pod with name" + - s" ${pod.getMetadata.getName} in namespace ${pod.getMetadata.getNamespace}") - PodUnknown(pod) - } - } - updatedPodSubscriber(state) - } - processedBatchSubscriber() - } - - private def isDeleted(pod: Pod): Boolean = pod.getMetadata.getDeletionTimestamp != null -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala index 2b0327deab9a2..59da53e3c3a3c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala @@ -18,16 +18,8 @@ package org.apache.spark.scheduler.cluster.k8s import io.fabric8.kubernetes.api.model.Pod -import org.apache.spark.deploy.k8s.Constants._ - sealed trait ExecutorPodState { def pod: Pod - - final def execId(): Long = pod - .getMetadata - .getLabels - .get(SPARK_EXECUTOR_ID_LABEL) - .toLong } case class PodRunning(pod: Pod) extends ExecutorPodState @@ -38,4 +30,6 @@ case class PodFailed(pod: Pod) extends ExecutorPodState case class PodDeleted(pod: Pod) extends ExecutorPodState +case class PodPending(pod: Pod) extends ExecutorPodState + case class PodUnknown(pod: Pod) extends ExecutorPodState diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 271ffb653e90b..bfbd9d01df199 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -18,20 +18,23 @@ package org.apache.spark.scheduler.cluster.k8s import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} -import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} +import io.fabric8.kubernetes.api.model.PodBuilder import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.mutable import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesConf import org.apache.spark.internal.Logging +import org.apache.spark.util.{Clock, Utils} private[spark] class ExecutorPodsAllocator( conf: SparkConf, executorBuilder: KubernetesExecutorBuilder, kubernetesClient: KubernetesClient, - eventQueue: ExecutorPodsEventQueue) extends Logging { + snapshotsStore: ExecutorPodsSnapshotsStore, + clock: Clock) extends Logging { private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) @@ -41,6 +44,8 @@ private[spark] class ExecutorPodsAllocator( private val podAllocationDelay = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) + private val podCreationTimeout = math.max(podAllocationDelay * 5, 60000) + private val kubernetesDriverPodName = conf .get(KUBERNETES_DRIVER_POD_NAME) .getOrElse(throw new SparkException("Must specify the driver pod name")) @@ -51,46 +56,64 @@ private[spark] class ExecutorPodsAllocator( // Use sets of ids instead of counters to be able to handle duplicate events. - // Executor IDs that have been requested from Kubernetes but are not running yet. - private val pendingExecutors = mutable.Set.empty[Long] - - // We could use CoarseGrainedSchedulerBackend#totalRegisteredExecutors here for tallying the - // executors that are running. But, here we choose instead to maintain all state within this - // class from the persecptive of the k8s API. Therefore whether or not this scheduler loop - // believes an executor is running is dictated by the K8s API rather than Spark's RPC events. - // We may need to consider where these perspectives may differ and which perspective should - // take precedence. - private val runningExecutors = mutable.Set.empty[Long] + // Executor IDs that have been requested from Kubernetes but have not been detected in any + // snapshot yet. Mapped to the timestamp when they were created. + private val newlyCreatedExecutors = mutable.Map.empty[Long, Long] def start(applicationId: String): Unit = { - eventQueue.addSubscriber( - podAllocationDelay, - new ExecutorPodBatchSubscriber( - processUpdatedPod(applicationId), - () => postProcessBatch(applicationId))) + snapshotsStore.addSubscriber(podAllocationDelay) { + processSnapshot(applicationId, _) + } } def setTotalExpectedExecutors(total: Int): Unit = totalExpectedExecutors.set(total) - private def processUpdatedPod(applicationId: String): PartialFunction[ExecutorPodState, Unit] = { - case running @ PodRunning(_) => - pendingExecutors -= running.execId() - runningExecutors += running.execId() - case completed @ (PodSucceeded(_) | PodDeleted(_) | PodFailed(_)) => - pendingExecutors -= completed.execId() - runningExecutors -= completed.execId() - case _ => - } + private def processSnapshot(applicationId: String, snapshot: ExecutorPodsSnapshot): Unit = { + snapshot.executorPods.filter { + case (_, PodPending(_)) | (_, PodUnknown(_)) => false + case _ => true + }.keys.foreach { + newlyCreatedExecutors -= _ + } - private def postProcessBatch(applicationId: String): Unit = { - val currentRunningExecutors = runningExecutors.size + // For all executors we've created against the API but have not seen in a snapshot + // yet - check the current time. If the current time has exceeded some threshold, + // assume that the pod was either never created (the API server never properly + // handled the creation request), or the API server created the pod but we missed + // both the creation and deletion events. In either case, delete the missing pod + // if possible, and mark such a pod to be rescheduled below. + (newlyCreatedExecutors.keySet -- snapshot.executorPods.keySet).foreach { execId => + // Wait for 1 minute + if (clock.getTimeMillis() - newlyCreatedExecutors(execId) > podCreationTimeout) { + logWarning(s"Executor with id $execId was not detected in the Kubernetes" + + " cluster after 1 minute despite the fact that a previous allocation attempt" + + " tried to create it. The executor may have been deleted but the application" + + " missed the deletion event.") + Utils.tryLogNonFatalError { + kubernetesClient + .pods() + .withLabel(SPARK_EXECUTOR_ID_LABEL, execId.toString) + .delete() + } + newlyCreatedExecutors -= execId + } + } + + val currentRunningExecutors = snapshot.executorPods.values.count { + case PodRunning(_) => true + case _ => false + } + val currentPendingExecutors = snapshot.executorPods.values.count { + case PodPending(_) => true + case _ => false + } val currentTotalExpectedExecutors = totalExpectedExecutors.get - if (pendingExecutors.isEmpty && currentRunningExecutors < currentTotalExpectedExecutors) { + if (newlyCreatedExecutors.isEmpty + && currentPendingExecutors == 0 + && currentRunningExecutors < currentTotalExpectedExecutors) { val numExecutorsToAllocate = math.min( currentTotalExpectedExecutors - currentRunningExecutors, podAllocationSize) logInfo(s"Going to request $numExecutorsToAllocate executors from Kubernetes.") - val newExecutorIds = mutable.Buffer.empty[Long] - val podsToAllocate = mutable.Buffer.empty[Pod] for ( _ <- 0 until numExecutorsToAllocate) { val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet() val executorConf = KubernetesConf.createExecutorConf( @@ -105,14 +128,15 @@ private[spark] class ExecutorPodsAllocator( .endSpec() .build() kubernetesClient.pods().create(podWithAttachedContainer) - pendingExecutors += newExecutorId + newlyCreatedExecutors(newExecutorId) = clock.getTimeMillis() } } else if (currentRunningExecutors >= currentTotalExpectedExecutors) { + // TODO handle edge cases if we end up with more running executors than expected. logDebug("Current number of running executors is equal to the number of requested" + " executors. Not scaling up further.") - } else if (pendingExecutors.nonEmpty) { - logDebug(s"Still waiting for ${pendingExecutors.size} executors to begin running before" + - " requesting for more executors.") + } else if (newlyCreatedExecutors.nonEmpty || currentPendingExecutors != 0) { + logDebug(s"Still waiting for ${newlyCreatedExecutors.size + currentPendingExecutors}" + + s" executors to begin running before requesting for more executors.") } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala similarity index 67% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala index e5a4b10e9422a..bc8205972f8c7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandler.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala @@ -20,44 +20,67 @@ import com.google.common.cache.Cache import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.JavaConverters._ +import scala.collection.mutable import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.scheduler.ExecutorExited import org.apache.spark.util.Utils -private[spark] class ExecutorPodsLifecycleEventHandler( +private[spark] class ExecutorPodsLifecycleManager( conf: SparkConf, executorBuilder: KubernetesExecutorBuilder, kubernetesClient: KubernetesClient, - podsEventQueue: ExecutorPodsEventQueue, + snapshotsStore: ExecutorPodsSnapshotsStore, // Use a best-effort to track which executors have been removed already. It's not generally // job-breaking if we remove executors more than once but it's ideal if we make an attempt // to avoid doing so. Expire cache entries so that this data structure doesn't grow beyond // bounds. removedExecutorsCache: Cache[java.lang.Long, java.lang.Long]) { - import ExecutorPodsLifecycleEventHandler._ + import ExecutorPodsLifecycleManager._ private val eventProcessingInterval = conf.get(KUBERNETES_EXECUTOR_EVENT_PROCESSING_INTERVAL) def start(schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { - podsEventQueue.addSubscriber( - eventProcessingInterval, - new ExecutorPodBatchSubscriber( - processUpdatedPod(schedulerBackend), - () => {})) + snapshotsStore.addSubscriber(eventProcessingInterval) { + onNextSnapshot(schedulerBackend, _) + } } - private def processUpdatedPod( - schedulerBackend: KubernetesClusterSchedulerBackend) - : PartialFunction[ExecutorPodState, Unit] = { - case deleted @ PodDeleted(pod) => - removeExecutorFromSpark(schedulerBackend, pod, deleted.execId()) - case errorOrSucceeded @ (PodFailed(_) | PodSucceeded(_)) => - removeExecutorFromK8s(errorOrSucceeded.pod) - removeExecutorFromSpark(schedulerBackend, errorOrSucceeded.pod, errorOrSucceeded.execId()) - case _ => + private def onNextSnapshot( + schedulerBackend: KubernetesClusterSchedulerBackend, + snapshot: ExecutorPodsSnapshot): Unit = { + val execIdsRemovedInThisRound = mutable.HashSet.empty[Long] + snapshot.executorPods.foreach { case (execId, state) => + state match { + case PodDeleted(pod) => + removeExecutorFromSpark(schedulerBackend, pod, execId) + execIdsRemovedInThisRound += execId + case errorOrSucceeded @ (PodFailed(_) | PodSucceeded(_)) => + removeExecutorFromK8s(errorOrSucceeded.pod) + removeExecutorFromSpark(schedulerBackend, errorOrSucceeded.pod, execId) + execIdsRemovedInThisRound += execId + case _ => + } + } + + // Reconcile the case where Spark claims to know about an executor but the corresponding pod + // is missing from the cluster. This would occur if we miss a deletion event and the pod + // transitions immediately from running io absent. + (schedulerBackend.getExecutorIds().map(_.toLong).toSet + -- snapshot.executorPods.keySet + -- execIdsRemovedInThisRound).foreach { missingExecutorId => + if (removedExecutorsCache.getIfPresent(missingExecutorId) == null) { + val exitReason = ExecutorExited( + UNKNOWN_EXIT_CODE, + exitCausedByApp = false, + s"The executor with ID $missingExecutorId was not found in the cluster but we didn't" + + s" get a reason why. Marking the executor as failed. The executor may have been" + + s" deleted but the driver missed the deletion event.") + schedulerBackend.doRemoveExecutor(missingExecutorId.toString, exitReason) + } + } } private def removeExecutorFromK8s(updatedPod: Pod): Unit = { @@ -117,7 +140,7 @@ private[spark] class ExecutorPodsLifecycleEventHandler( } } -private object ExecutorPodsLifecycleEventHandler { +private object ExecutorPodsLifecycleManager { val UNKNOWN_EXIT_CODE = -1 } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala index e41ec6539bc09..6a664f4edf612 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala @@ -29,7 +29,7 @@ import org.apache.spark.util.ThreadUtils private[spark] class ExecutorPodsPollingEventSource( conf: SparkConf, kubernetesClient: KubernetesClient, - eventQueue: ExecutorPodsEventQueue, + snapshotsStore: ExecutorPodsSnapshotsStore, pollingExecutor: ScheduledExecutorService) { private val pollingInterval = conf.get(KUBERNETES_EXECUTOR_API_POLLING_INTERVAL) @@ -52,14 +52,13 @@ private[spark] class ExecutorPodsPollingEventSource( private class PollRunnable(applicationId: String) extends Runnable { override def run(): Unit = { - kubernetesClient + snapshotsStore.replaceSnapshot(kubernetesClient .pods() .withLabel(SPARK_APP_ID_LABEL, applicationId) .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .list() .getItems - .asScala - .foreach(eventQueue.enqueue) + .asScala) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala new file mode 100644 index 0000000000000..26be918043412 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala @@ -0,0 +1,74 @@ +/* + * 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 io.fabric8.kubernetes.api.model.Pod + +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.internal.Logging + +/** + * An immutable view of the current executor pods that are running in the cluster. + */ +private[spark] case class ExecutorPodsSnapshot(executorPods: Map[Long, ExecutorPodState]) { + + import ExecutorPodsSnapshot._ + + def withUpdate(updatedPod: Pod): ExecutorPodsSnapshot = { + val newExecutorPods = executorPods ++ toStatesByExecutorId(Seq(updatedPod)) + new ExecutorPodsSnapshot(newExecutorPods) + } +} + +object ExecutorPodsSnapshot extends Logging { + + def apply(executorPods: Seq[Pod]): ExecutorPodsSnapshot = { + ExecutorPodsSnapshot(toStatesByExecutorId(executorPods)) + } + + def apply(): ExecutorPodsSnapshot = ExecutorPodsSnapshot(Map.empty[Long, ExecutorPodState]) + + private def toStatesByExecutorId(executorPods: Seq[Pod]): Map[Long, ExecutorPodState] = { + executorPods.map { pod => + (pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL).toLong, toState(pod)) + }.toMap + } + + private def toState(pod: Pod): ExecutorPodState = { + if (isDeleted(pod)) { + PodDeleted(pod) + } else { + val phase = pod.getStatus.getPhase.toLowerCase + phase match { + case "pending" => + PodPending(pod) + case "running" => + PodRunning(pod) + case "failed" => + PodFailed(pod) + case "succeeded" => + PodSucceeded(pod) + case _ => + logWarning(s"Received unknown phase $phase for executor pod with name" + + s" ${pod.getMetadata.getName} in namespace ${pod.getMetadata.getNamespace}") + PodUnknown(pod) + } + } + } + + private def isDeleted(pod: Pod): Boolean = pod.getMetadata.getDeletionTimestamp != null +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala similarity index 78% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala index da71678856f42..37c1c7f6b6213 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueue.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala @@ -18,12 +18,15 @@ package org.apache.spark.scheduler.cluster.k8s import io.fabric8.kubernetes.api.model.Pod -private[spark] trait ExecutorPodsEventQueue { +private[spark] trait ExecutorPodsSnapshotsStore { - def addSubscriber(processBatchIntervalMillis: Long, subscriber: ExecutorPodBatchSubscriber): Unit + def addSubscriber + (processBatchIntervalMillis: Long) + (onNextSnapshot: ExecutorPodsSnapshot => Unit) def stop(): Unit - def enqueue(updatedPod: Pod): Unit + def updatePod(updatedPod: Pod): Unit + def replaceSnapshot(newSnapshot: Seq[Pod]): Unit } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala similarity index 65% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala index 598c727238beb..5abdd4f18f1e5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala @@ -24,22 +24,29 @@ import io.reactivex.disposables.Disposable import io.reactivex.functions.Consumer import io.reactivex.schedulers.Schedulers import io.reactivex.subjects.PublishSubject +import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.util.{ThreadUtils, Utils} -private[spark] class ExecutorPodsEventQueueImpl( - bufferEventsExecutor: ScheduledExecutorService, executeSubscriptionsExecutor: ExecutorService) - extends ExecutorPodsEventQueue { +private[spark] class ExecutorPodsSnapshotsStoreImpl( + bufferSnapshotsExecutor: ScheduledExecutorService, + executeSubscriptionsExecutor: ExecutorService) + extends ExecutorPodsSnapshotsStore { - private val eventsObservable = PublishSubject.create[Pod]() + private val SNAPSHOT_LOCK = new Object() + + private val snapshotsObservable = PublishSubject.create[ExecutorPodsSnapshot]() private val observedDisposables = mutable.Buffer.empty[Disposable] - def addSubscriber( - processBatchIntervalMillis: Long, - subscriber: ExecutorPodBatchSubscriber): Unit = { - observedDisposables += eventsObservable + @GuardedBy("SNAPSHOT_LOCK") + private var currentSnapshot = ExecutorPodsSnapshot() + + override def addSubscriber( + processBatchIntervalMillis: Long) + (subscriber: ExecutorPodsSnapshot => Unit): Unit = { + observedDisposables += snapshotsObservable // Group events in the time window given by the caller. These buffers are then sent // to the caller's lambda at the given interval, with the pod updates that occurred // in that given interval. @@ -48,29 +55,37 @@ private[spark] class ExecutorPodsEventQueueImpl( TimeUnit.MILLISECONDS, // For testing - specifically use the given scheduled executor service to trigger // buffer boundaries. Allows us to inject a deterministic scheduler here. - Schedulers.from(bufferEventsExecutor)) + Schedulers.from(bufferSnapshotsExecutor)) // Trigger an event cycle immediately. Not strictly required to be fully correct, but // in particular the pod allocator should try to request executors immediately instead // of waiting for one pod allocation delay. - .startWith(Lists.newArrayList[Pod]()) + .startWith(Lists.newArrayList(ExecutorPodsSnapshot())) // Force all triggered events - both the initial event above and the buffered ones in // the following time windows - to execute asynchronously to this call's thread. .observeOn(Schedulers.from(executeSubscriptionsExecutor)) - .subscribe(toReactivexConsumer { pods: java.util.List[Pod] => + .subscribe(toReactivexConsumer { snapshots: java.util.List[ExecutorPodsSnapshot] => Utils.tryLogNonFatalError { - subscriber.onNextBatch(pods.asScala) + snapshots.asScala.foreach(subscriber) } }) } - def stop(): Unit = { + override def stop(): Unit = { observedDisposables.foreach(_.dispose()) - eventsObservable.onComplete() - ThreadUtils.shutdown(bufferEventsExecutor) + snapshotsObservable.onComplete() + ThreadUtils.shutdown(bufferSnapshotsExecutor) ThreadUtils.shutdown(executeSubscriptionsExecutor) } - def enqueue(updatedPod: Pod): Unit = eventsObservable.onNext(updatedPod) + override def updatePod(updatedPod: Pod): Unit = SNAPSHOT_LOCK.synchronized { + currentSnapshot = currentSnapshot.withUpdate(updatedPod) + snapshotsObservable.onNext(currentSnapshot) + } + + override def replaceSnapshot(newSnapshot: Seq[Pod]): Unit = SNAPSHOT_LOCK.synchronized { + currentSnapshot = ExecutorPodsSnapshot(newSnapshot) + snapshotsObservable.onNext(currentSnapshot) + } private def toReactivexConsumer[T](consumer: T => Unit): Consumer[T] = { new Consumer[T] { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala index f307375b01fda..b82ad338b9132 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala @@ -27,7 +27,8 @@ import org.apache.spark.internal.Logging import org.apache.spark.util.Utils private[spark] class ExecutorPodsWatchEventSource( - podsEventQueue: ExecutorPodsEventQueue, kubernetesClient: KubernetesClient) extends Logging { + snapshotsStore: ExecutorPodsSnapshotsStore, + kubernetesClient: KubernetesClient) extends Logging { private var watchConnection: Closeable = _ @@ -50,7 +51,7 @@ private[spark] class ExecutorPodsWatchEventSource( private class ExecutorPodsWatcher extends Watcher[Pod] { override def eventReceived(action: Action, pod: Pod): Unit = { - podsEventQueue.enqueue(pod) + snapshotsStore.updatePod(pod) } override def onClose(e: KubernetesClientException): Unit = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index ee211d24dc76e..0ae7e274c0e36 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -28,7 +28,7 @@ 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 +import org.apache.spark.util.{SystemClock, ThreadUtils} private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { @@ -61,40 +61,40 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( "kubernetes-executor-requests") - val bufferEventsExecutor = ThreadUtils - .newDaemonSingleThreadScheduledExecutor("kubernetes-executor-pods-event-buffer") - val executeEventSubscribersExecutor = ThreadUtils - .newDaemonCachedThreadPool("kubernetes-executor-pods-event-handlers") - val eventQueue = new ExecutorPodsEventQueueImpl( - bufferEventsExecutor, executeEventSubscribersExecutor) + val bufferSnapshotsExecutor = ThreadUtils + .newDaemonSingleThreadScheduledExecutor("kubernetes-executor-snapshots-buffer") + val executeSubscribersExecutor = ThreadUtils + .newDaemonCachedThreadPool("kubernetes-executor-snapshots-handlers") + val snapshotsStore = new ExecutorPodsSnapshotsStoreImpl( + bufferSnapshotsExecutor, executeSubscribersExecutor) val removedExecutorsCache = CacheBuilder.newBuilder() .expireAfterWrite(3, TimeUnit.MINUTES) .build[java.lang.Long, java.lang.Long]() - val executorPodsLifecycleEventHandler = new ExecutorPodsLifecycleEventHandler( + val executorPodsLifecycleEventHandler = new ExecutorPodsLifecycleManager( sc.conf, new KubernetesExecutorBuilder(), kubernetesClient, - eventQueue, + snapshotsStore, removedExecutorsCache) val executorPodsAllocator = new ExecutorPodsAllocator( - sc.conf, new KubernetesExecutorBuilder(), kubernetesClient, eventQueue) + sc.conf, new KubernetesExecutorBuilder(), kubernetesClient, snapshotsStore, new SystemClock()) val podsWatchEventSource = new ExecutorPodsWatchEventSource( - eventQueue, + snapshotsStore, kubernetesClient) val eventsPollingExecutor = ThreadUtils.newDaemonSingleThreadScheduledExecutor( "kubernetes-executor-pod-polling-sync") val podsPollingEventSource = new ExecutorPodsPollingEventSource( - sc.conf, kubernetesClient, eventQueue, eventsPollingExecutor) + sc.conf, kubernetesClient, snapshotsStore, eventsPollingExecutor) new KubernetesClusterSchedulerBackend( scheduler.asInstanceOf[TaskSchedulerImpl], sc.env.rpcEnv, kubernetesClient, requestExecutorsService, - eventQueue, + snapshotsStore, executorPodsAllocator, executorPodsLifecycleEventHandler, podsWatchEventSource, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 5f00fda396297..1908ce0ce0d6a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -32,9 +32,9 @@ private[spark] class KubernetesClusterSchedulerBackend( rpcEnv: RpcEnv, kubernetesClient: KubernetesClient, requestExecutorsService: ExecutorService, - eventQueue: ExecutorPodsEventQueue, + snapshotsStore: ExecutorPodsSnapshotsStore, podAllocator: ExecutorPodsAllocator, - lifecycleEventHandler: ExecutorPodsLifecycleEventHandler, + lifecycleEventHandler: ExecutorPodsLifecycleManager, watchEvents: ExecutorPodsWatchEventSource, pollEvents: ExecutorPodsPollingEventSource) extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { @@ -71,7 +71,7 @@ private[spark] class KubernetesClusterSchedulerBackend( super.stop() Utils.tryLogNonFatalError { - eventQueue.stop() + snapshotsStore.stop() } Utils.tryLogNonFatalError { @@ -107,6 +107,10 @@ private[spark] class KubernetesClusterSchedulerBackend( totalRegisteredExecutors.get() >= initialExecutors * minRegisteredRatio } + override def getExecutorIds(): Seq[String] = synchronized { + super.getExecutorIds() + } + override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { kubernetesClient.pods() .withLabel(SPARK_APP_ID_LABEL, applicationId()) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala similarity index 57% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala index f2912e4efec28..3c5b62441b210 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsEventQueue.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala @@ -19,25 +19,34 @@ package org.apache.spark.scheduler.cluster.k8s import io.fabric8.kubernetes.api.model.Pod import scala.collection.mutable -class DeterministicExecutorPodsEventQueue extends ExecutorPodsEventQueue { +class DeterministicExecutorPodsSnapshotsStore extends ExecutorPodsSnapshotsStore { - private val eventBuffer = mutable.Buffer.empty[Pod] - private val subscribers = mutable.Buffer.empty[ExecutorPodBatchSubscriber] + private val snapshotsBuffer = mutable.Buffer.empty[ExecutorPodsSnapshot] + private val subscribers = mutable.Buffer.empty[ExecutorPodsSnapshot => Unit] - override def addSubscriber( - processBatchIntervalMillis: Long, - subscriber: ExecutorPodBatchSubscriber): Unit = { + private var currentSnapshot = ExecutorPodsSnapshot() + + override def addSubscriber + (processBatchIntervalMillis: Long) + (subscriber: ExecutorPodsSnapshot => Unit): Unit = { subscribers += subscriber } override def stop(): Unit = {} - override def enqueue(updatedPod: Pod): Unit = eventBuffer += updatedPod - def notifySubscribers(): Unit = { subscribers.foreach { subscriber => - subscriber.onNextBatch(eventBuffer) + snapshotsBuffer.foreach(subscriber) } - eventBuffer.clear() + snapshotsBuffer.clear() + } + override def updatePod(updatedPod: Pod): Unit = { + currentSnapshot = currentSnapshot.withUpdate(updatedPod) + snapshotsBuffer += currentSnapshot + } + + override def replaceSnapshot(newSnapshot: Seq[Pod]): Unit = { + currentSnapshot = ExecutorPodsSnapshot(newSnapshot) + snapshotsBuffer += currentSnapshot } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala index df33015b4cd73..c6b667ed85e8c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala @@ -55,6 +55,14 @@ object ExecutorLifecycleTestUtils { .build() } + def pendingExecutor(executorId: Long): Pod = { + new PodBuilder(podWithAttachedContainerForId(executorId)) + .editOrNewStatus() + .withPhase("pending") + .endStatus() + .build() + } + def runningExecutor(executorId: Long): Pod = { new PodBuilder(podWithAttachedContainerForId(executorId)) .editOrNewStatus() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriberSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriberSuite.scala deleted file mode 100644 index 6f6bcc04c0866..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodBatchSubscriberSuite.scala +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.scheduler.cluster.k8s - -import java.util.concurrent.atomic.AtomicInteger - -import io.fabric8.kubernetes.api.model.Pod - -import org.apache.spark.SparkFunSuite - -private[spark] class ExecutorPodBatchSubscriberSuite extends SparkFunSuite { - - test("Check the correctness of states.") { - val statesDetectionFunction = new StatesDetectionFunction() - val subscriber = new ExecutorPodBatchSubscriber(statesDetectionFunction, () => {}) - val running = ExecutorLifecycleTestUtils.runningExecutor(1) - val failed = ExecutorLifecycleTestUtils.failedExecutorWithoutDeletion(2) - val deleted = ExecutorLifecycleTestUtils.deletedExecutor(3) - val succeeded = ExecutorLifecycleTestUtils.succeededExecutor(4) - val unknown = ExecutorLifecycleTestUtils.unknownExecutor(5) - subscriber.onNextBatch(Seq(running, failed, deleted, succeeded, unknown)) - assert(statesDetectionFunction.podRunning === PodRunning(running)) - assert(statesDetectionFunction.podFailed === PodFailed(failed)) - assert(statesDetectionFunction.podDeleted === PodDeleted(deleted)) - assert(statesDetectionFunction.podSucceeded === PodSucceeded(succeeded)) - assert(statesDetectionFunction.podUnknown === PodUnknown(unknown)) - } - - test("Invoke post batch processor after every batch.") { - val batchProcessedCounter = new AtomicInteger(0) - val subscriber = new ExecutorPodBatchSubscriber( - new StatesDetectionFunction(), () => batchProcessedCounter.getAndIncrement()) - subscriber.onNextBatch(Seq.empty[Pod]) - assert(batchProcessedCounter.get === 1) - subscriber.onNextBatch(Seq.empty[Pod]) - assert(batchProcessedCounter.get === 2) - } - - private class StatesDetectionFunction extends PartialFunction[ExecutorPodState, Unit] { - var podRunning: PodRunning = _ - var podFailed: PodFailed = _ - var podDeleted: PodDeleted = _ - var podSucceeded: PodSucceeded = _ - var podUnknown: PodUnknown = _ - - override def isDefinedAt(state: ExecutorPodState): Boolean = { - state match { - case PodRunning(_) | PodFailed(_) | PodDeleted(_) | PodSucceeded(_) | PodUnknown(_) => true - case _ => false - } - } - - override def apply(state: ExecutorPodState): Unit = { - state match { - case running @ PodRunning(_) => podRunning = running - case failed @ PodFailed(_) => podFailed = failed - case deleted @ PodDeleted(_) => podDeleted = deleted - case succeeded @ PodSucceeded(_) => podSucceeded = succeeded - case unknown @ PodUnknown(_) => podUnknown = unknown - case other @ _ => throw new IllegalArgumentException(s"Unknown state $other") - } - } - } - -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 33a7246aca178..d6008824ad2b7 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._ +import org.apache.spark.util.ManualClock class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { @@ -49,6 +50,10 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { private val conf = new SparkConf().set(KUBERNETES_DRIVER_POD_NAME, driverPodName) private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) + private val podAllocationDelay = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) + private val podCreationTimeout = math.max(podAllocationDelay * 5, 60000L) + + private var waitForExecutorPodsClock: ManualClock = _ @Mock private var kubernetesClient: KubernetesClient = _ @@ -56,13 +61,16 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { @Mock private var podOperations: PODS = _ + @Mock + private var labeledPods: LABELED_PODS = _ + @Mock private var driverPodOperations: PodResource[Pod, DoneablePod] = _ @Mock private var executorBuilder: KubernetesExecutorBuilder = _ - private var eventQueue: DeterministicExecutorPodsEventQueue = _ + private var snapshotsStore: DeterministicExecutorPodsSnapshotsStore = _ private var podsAllocatorUnderTest: ExecutorPodsAllocator = _ @@ -73,55 +81,69 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { when(driverPodOperations.get).thenReturn(driverPod) when(executorBuilder.buildFromFeatures(kubernetesConfWithCorrectFields())) .thenAnswer(executorPodAnswer()) - eventQueue = new DeterministicExecutorPodsEventQueue() + snapshotsStore = new DeterministicExecutorPodsSnapshotsStore() + waitForExecutorPodsClock = new ManualClock(0L) podsAllocatorUnderTest = new ExecutorPodsAllocator( - conf, executorBuilder, kubernetesClient, eventQueue) + conf, executorBuilder, kubernetesClient, snapshotsStore, waitForExecutorPodsClock) podsAllocatorUnderTest.start(TEST_SPARK_APP_ID) } test("Initially request executors in batches. Do not request another batch if the" + " first has not finished.") { podsAllocatorUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) - eventQueue.notifySubscribers() + snapshotsStore.replaceSnapshot(Seq.empty[Pod]) + snapshotsStore.notifySubscribers() for (nextId <- 1 to podAllocationSize) { - verify(podOperations).create( - podWithAttachedContainerForId(nextId)) + verify(podOperations).create(podWithAttachedContainerForId(nextId)) } - verify(podOperations, never()).create( - podWithAttachedContainerForId(podAllocationSize + 1)) + verify(podOperations, never()).create(podWithAttachedContainerForId(podAllocationSize + 1)) } test("Request executors in batches. Allow another batch to be requested if" + " all pending executors start running.") { podsAllocatorUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) - eventQueue.notifySubscribers() + snapshotsStore.replaceSnapshot(Seq.empty[Pod]) + snapshotsStore.notifySubscribers() for (execId <- 1 until podAllocationSize) { - eventQueue.enqueue(runningExecutor(execId)) + snapshotsStore.updatePod(runningExecutor(execId)) } - eventQueue.notifySubscribers() - verify(podOperations, never()).create( - podWithAttachedContainerForId(podAllocationSize + 1)) - eventQueue.enqueue( - runningExecutor(podAllocationSize)) - eventQueue.notifySubscribers() + snapshotsStore.notifySubscribers() + verify(podOperations, never()).create(podWithAttachedContainerForId(podAllocationSize + 1)) + snapshotsStore.updatePod(runningExecutor(podAllocationSize)) + snapshotsStore.notifySubscribers() verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) - eventQueue.notifySubscribers() + snapshotsStore.updatePod(runningExecutor(podAllocationSize)) + snapshotsStore.notifySubscribers() verify(podOperations, times(podAllocationSize + 1)).create(any(classOf[Pod])) } test("When a current batch reaches error states immediately, re-request" + " them on the next batch.") { podsAllocatorUnderTest.setTotalExpectedExecutors(podAllocationSize) - eventQueue.notifySubscribers() + snapshotsStore.replaceSnapshot(Seq.empty[Pod]) + snapshotsStore.notifySubscribers() for (execId <- 1 until podAllocationSize) { - eventQueue.enqueue(runningExecutor(execId)) + snapshotsStore.updatePod(runningExecutor(execId)) } val failedPod = failedExecutorWithoutDeletion(podAllocationSize) - eventQueue.enqueue(failedPod) - eventQueue.notifySubscribers() + snapshotsStore.updatePod(failedPod) + snapshotsStore.notifySubscribers() verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) } + test("When an executor is requested but the API does not report it in 1 minute, retry" + + " requesting that executor.") { + podsAllocatorUnderTest.setTotalExpectedExecutors(1) + snapshotsStore.replaceSnapshot(Seq.empty[Pod]) + snapshotsStore.notifySubscribers() + snapshotsStore.replaceSnapshot(Seq.empty[Pod]) + waitForExecutorPodsClock.setTime(podCreationTimeout + 1) + when(podOperations.withLabel(SPARK_EXECUTOR_ID_LABEL, "1")).thenReturn(labeledPods) + snapshotsStore.notifySubscribers() + verify(labeledPods).delete() + verify(podOperations).create(podWithAttachedContainerForId(2)) + } + private def executorPodAnswer(): Answer[SparkPod] = { new Answer[SparkPod] { override def answer(invocation: InvocationOnMock): SparkPod = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala deleted file mode 100644 index ecdeae65e1067..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsEventQueueSuite.scala +++ /dev/null @@ -1,124 +0,0 @@ -/* - * 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.TimeUnit -import java.util.concurrent.atomic.AtomicReference - -import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} -import org.jmock.lib.concurrent.DeterministicScheduler -import org.mockito.Matchers.any -import org.mockito.Mockito.{mock, when} -import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer -import org.scalatest.BeforeAndAfter -import scala.collection.mutable - -import org.apache.spark.SparkFunSuite - -class ExecutorPodsEventQueueSuite extends SparkFunSuite with BeforeAndAfter { - - private var eventBufferScheduler: DeterministicScheduler = _ - private var executeSubscriptionsExecutor: DeterministicScheduler = _ - private var eventQueueUnderTest: ExecutorPodsEventQueueImpl = _ - - before { - eventBufferScheduler = new DeterministicScheduler() - executeSubscriptionsExecutor = new DeterministicScheduler() - eventQueueUnderTest = new ExecutorPodsEventQueueImpl( - eventBufferScheduler, - executeSubscriptionsExecutor) - } - - test("Subscribers get notified of events periodically.") { - val receivedEvents1 = mutable.Buffer.empty[Pod] - val receivedEvents2 = mutable.Buffer.empty[Pod] - eventQueueUnderTest.addSubscriber(1000, testBatchSubscriber(receivedEvents1)) - eventQueueUnderTest.addSubscriber(2000, testBatchSubscriber(receivedEvents2)) - - pushPodWithIndex(1) - assert(receivedEvents1.isEmpty) - assert(receivedEvents2.isEmpty) - // Force time to move forward so that the buffer is emitted, scheduling the - // processing task on the subscription executor... - eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) - // ... then actually execute the subscribers. - executeSubscriptionsExecutor.runUntilIdle() - assertIndicesMatch(receivedEvents1, 1) - assert(receivedEvents2.isEmpty) - eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) - executeSubscriptionsExecutor.runUntilIdle() - assertIndicesMatch(receivedEvents2, 1) - pushPodWithIndex(2) - pushPodWithIndex(3) - eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) - executeSubscriptionsExecutor.runUntilIdle() - assertIndicesMatch(receivedEvents1, 1, 2, 3) - assertIndicesMatch(receivedEvents2, 1) - eventBufferScheduler.tick(2000, TimeUnit.MILLISECONDS) - executeSubscriptionsExecutor.runUntilIdle() - assertIndicesMatch(receivedEvents1, 1, 2, 3) - assertIndicesMatch(receivedEvents2, 1, 2, 3) - } - - test("Even without sending events, initially receive an empty buffer.") { - val receivedInitialBuffer = new AtomicReference[Iterable[Pod]](null) - eventQueueUnderTest.addSubscriber(1000, testSetBufferSubscriber(receivedInitialBuffer)) - assert(receivedInitialBuffer.get == null) - executeSubscriptionsExecutor.runUntilIdle() - assert(receivedInitialBuffer.get != null) - } - - private def assertIndicesMatch(buffer: mutable.Buffer[Pod], indices: Int*): Unit = { - assert(buffer === indices.map(podWithIndex)) - } - - private def pushPodWithIndex(index: Int): Unit = - eventQueueUnderTest.enqueue(podWithIndex(index)) - - private def podWithIndex(index: Int): Pod = - new PodBuilder() - .editOrNewMetadata() - .withName(s"pod-$index") - .endMetadata() - .build() - - private def testBatchSubscriber(eventBuffer: mutable.Buffer[Pod]): ExecutorPodBatchSubscriber = { - val subscriber = mock(classOf[ExecutorPodBatchSubscriber]) - when(subscriber.onNextBatch(any(classOf[Iterable[Pod]]))) - .thenAnswer(new Answer[Unit] { - override def answer(invocationOnMock: InvocationOnMock): Unit = { - val pods = invocationOnMock.getArgumentAt(0, classOf[Iterable[Pod]]) - eventBuffer ++= pods - } - }) - subscriber - } - - private def testSetBufferSubscriber( - eventBuffer: AtomicReference[Iterable[Pod]]): ExecutorPodBatchSubscriber = { - val subscriber = mock(classOf[ExecutorPodBatchSubscriber]) - when(subscriber.onNextBatch(any(classOf[Iterable[Pod]]))) - .thenAnswer(new Answer[Unit] { - override def answer(invocationOnMock: InvocationOnMock): Unit = { - val pods = invocationOnMock.getArgumentAt(0, classOf[Iterable[Pod]]) - eventBuffer.set(pods) - } - }) - subscriber - } -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala similarity index 76% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala index 0273ab90241fa..562ace9f49d4d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleEventHandlerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.deploy.k8s.Fabric8Aliases._ import org.apache.spark.scheduler.ExecutorExited import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._ -class ExecutorPodsLifecycleEventHandlerSuite extends SparkFunSuite with BeforeAndAfter { +class ExecutorPodsLifecycleManagerSuite extends SparkFunSuite with BeforeAndAfter { private var namedExecutorPods: mutable.Map[String, PodResource[Pod, DoneablePod]] = _ @@ -50,29 +50,30 @@ class ExecutorPodsLifecycleEventHandlerSuite extends SparkFunSuite with BeforeAn @Mock private var schedulerBackend: KubernetesClusterSchedulerBackend = _ - private var eventQueue: DeterministicExecutorPodsEventQueue = _ - private var eventHandlerUnderTest: ExecutorPodsLifecycleEventHandler = _ + private var snapshotsStore: DeterministicExecutorPodsSnapshotsStore = _ + private var eventHandlerUnderTest: ExecutorPodsLifecycleManager = _ before { MockitoAnnotations.initMocks(this) val removedExecutorsCache = CacheBuilder.newBuilder().build[java.lang.Long, java.lang.Long] - eventQueue = new DeterministicExecutorPodsEventQueue() + snapshotsStore = new DeterministicExecutorPodsSnapshotsStore() namedExecutorPods = mutable.Map.empty[String, PodResource[Pod, DoneablePod]] + when(schedulerBackend.getExecutorIds()).thenReturn(Seq.empty[String]) when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(any(classOf[String]))).thenAnswer(namedPodsAnswer()) - eventHandlerUnderTest = new ExecutorPodsLifecycleEventHandler( + eventHandlerUnderTest = new ExecutorPodsLifecycleManager( new SparkConf(), executorBuilder, kubernetesClient, - eventQueue, + snapshotsStore, removedExecutorsCache) eventHandlerUnderTest.start(schedulerBackend) } test("When an executor reaches error states immediately, remove from the scheduler backend.") { val failedPod = failedExecutorWithoutDeletion(1) - eventQueue.enqueue(failedPod) - eventQueue.notifySubscribers() + snapshotsStore.updatePod(failedPod) + snapshotsStore.notifySubscribers() val msg = exitReasonMessage(1, failedPod) val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) verify(schedulerBackend).doRemoveExecutor("1", expectedLossReason) @@ -81,15 +82,27 @@ class ExecutorPodsLifecycleEventHandlerSuite extends SparkFunSuite with BeforeAn test("Don't remove executors twice from Spark but remove from K8s repeatedly.") { val failedPod = failedExecutorWithoutDeletion(1) - eventQueue.enqueue(failedPod) - eventQueue.enqueue(failedPod) - eventQueue.notifySubscribers() + snapshotsStore.updatePod(failedPod) + snapshotsStore.updatePod(failedPod) + snapshotsStore.notifySubscribers() val msg = exitReasonMessage(1, failedPod) val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) verify(schedulerBackend, times(1)).doRemoveExecutor("1", expectedLossReason) verify(namedExecutorPods(failedPod.getMetadata.getName), times(2)).delete() } + test("When the scheduler backend lists executor ids that aren't present in the cluster," + + " remove those executors from Spark.") { + when(schedulerBackend.getExecutorIds()).thenReturn(Seq("1")) + val msg = s"The executor with ID 1 was not found in the cluster but we didn't" + + s" get a reason why. Marking the executor as failed. The executor may have been" + + s" deleted but the driver missed the deletion event." + val expectedLossReason = ExecutorExited(-1, exitCausedByApp = false, msg) + snapshotsStore.replaceSnapshot(Seq.empty[Pod]) + snapshotsStore.notifySubscribers() + verify(schedulerBackend).doRemoveExecutor("1", expectedLossReason) + } + private def exitReasonMessage(failedExecutorId: Int, failedPod: Pod): String = { s""" |The executor with id $failedExecutorId exited with exit code 1. diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala index 36a945916f4da..50a2d9b0aacf4 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala @@ -50,7 +50,7 @@ class ExecutorPodsPollingEventSourceSuite extends SparkFunSuite with BeforeAndAf private var executorRoleLabeledPods: LABELED_PODS = _ @Mock - private var eventQueue: ExecutorPodsEventQueue = _ + private var eventQueue: ExecutorPodsSnapshotsStore = _ private var pollingExecutor: DeterministicScheduler = _ private var pollingSourceUnderTest: ExecutorPodsPollingEventSource = _ @@ -79,8 +79,7 @@ class ExecutorPodsPollingEventSourceSuite extends SparkFunSuite with BeforeAndAf runningExecutor(2)) .build()) pollingExecutor.tick(pollingInterval, TimeUnit.MILLISECONDS) - verify(eventQueue).enqueue(runningExecutor(1)) - verify(eventQueue).enqueue(runningExecutor(2)) + verify(eventQueue).replaceSnapshot(Seq(runningExecutor(1), runningExecutor(2))) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotSuite.scala new file mode 100644 index 0000000000000..70e19c904eddb --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotSuite.scala @@ -0,0 +1,60 @@ +/* + * 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 org.apache.spark.SparkFunSuite +import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._ + +class ExecutorPodsSnapshotSuite extends SparkFunSuite { + + test("States are interpreted correctly from pod metadata.") { + val pods = Seq( + pendingExecutor(0), + runningExecutor(1), + succeededExecutor(2), + failedExecutorWithoutDeletion(3), + deletedExecutor(4), + unknownExecutor(5)) + val snapshot = ExecutorPodsSnapshot(pods) + assert(snapshot.executorPods === + Map( + 0L -> PodPending(pods(0)), + 1L -> PodRunning(pods(1)), + 2L -> PodSucceeded(pods(2)), + 3L -> PodFailed(pods(3)), + 4L -> PodDeleted(pods(4)), + 5L -> PodUnknown(pods(5)))) + } + + test("Updates add new pods for non-matching ids and edit existing pods for matching ids") { + val originalPods = Seq( + pendingExecutor(0), + runningExecutor(1)) + val originalSnapshot = ExecutorPodsSnapshot(originalPods) + val snapshotWithUpdatedPod = originalSnapshot.withUpdate(succeededExecutor(1)) + assert(snapshotWithUpdatedPod.executorPods === + Map( + 0L -> PodPending(originalPods(0)), + 1L -> PodSucceeded(succeededExecutor(1)))) + val snapshotWithNewPod = snapshotWithUpdatedPod.withUpdate(pendingExecutor(2)) + assert(snapshotWithNewPod.executorPods === + Map( + 0L -> PodPending(originalPods(0)), + 1L -> PodSucceeded(succeededExecutor(1)), + 2L -> PodPending(pendingExecutor(2)))) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala new file mode 100644 index 0000000000000..708af04ed0243 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala @@ -0,0 +1,146 @@ +/* + * 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.TimeUnit +import java.util.concurrent.atomic.AtomicReference + +import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} +import org.jmock.lib.concurrent.DeterministicScheduler +import org.scalatest.BeforeAndAfter +import scala.collection.mutable + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.Constants._ + +class ExecutorPodsSnapshotsStoreSuite extends SparkFunSuite with BeforeAndAfter { + + private var eventBufferScheduler: DeterministicScheduler = _ + private var executeSubscriptionsExecutor: DeterministicScheduler = _ + private var eventQueueUnderTest: ExecutorPodsSnapshotsStoreImpl = _ + + before { + eventBufferScheduler = new DeterministicScheduler() + executeSubscriptionsExecutor = new DeterministicScheduler() + eventQueueUnderTest = new ExecutorPodsSnapshotsStoreImpl( + eventBufferScheduler, + executeSubscriptionsExecutor) + } + + test("Subscribers get notified of events periodically.") { + val receivedSnapshots1 = mutable.Buffer.empty[ExecutorPodsSnapshot] + val receivedSnapshots2 = mutable.Buffer.empty[ExecutorPodsSnapshot] + eventQueueUnderTest.addSubscriber(1000) { + receivedSnapshots1 += _ + } + eventQueueUnderTest.addSubscriber(2000) { + receivedSnapshots2 += _ + } + + pushPodWithIndex(1) + assert(receivedSnapshots1.isEmpty) + assert(receivedSnapshots2.isEmpty) + // Force time to move forward so that the buffer is emitted, scheduling the + // processing task on the subscription executor... + eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) + // ... then actually execute the subscribers. + executeSubscriptionsExecutor.runUntilIdle() + + assert(receivedSnapshots1 === Seq( + ExecutorPodsSnapshot(), + ExecutorPodsSnapshot(Seq(podWithIndex(1))))) + assert(receivedSnapshots2 === Seq(ExecutorPodsSnapshot())) + + eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) + executeSubscriptionsExecutor.runUntilIdle() + + // Don't repeat snapshots + assert(receivedSnapshots1 === Seq( + ExecutorPodsSnapshot(), + ExecutorPodsSnapshot(Seq(podWithIndex(1))))) + assert(receivedSnapshots2 === Seq( + ExecutorPodsSnapshot(), + ExecutorPodsSnapshot(Seq(podWithIndex(1)))) + ) + pushPodWithIndex(2) + pushPodWithIndex(3) + eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) + executeSubscriptionsExecutor.runUntilIdle() + + assert(receivedSnapshots1 === Seq( + ExecutorPodsSnapshot(), + ExecutorPodsSnapshot(Seq(podWithIndex(1))), + ExecutorPodsSnapshot(Seq(podWithIndex(1), podWithIndex(2))), + ExecutorPodsSnapshot(Seq(podWithIndex(1), podWithIndex(2), podWithIndex(3))))) + assert(receivedSnapshots2 === Seq( + ExecutorPodsSnapshot(), + ExecutorPodsSnapshot(Seq(podWithIndex(1))))) + + eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) + executeSubscriptionsExecutor.runUntilIdle() + assert(receivedSnapshots1 === Seq( + ExecutorPodsSnapshot(), + ExecutorPodsSnapshot(Seq(podWithIndex(1))), + ExecutorPodsSnapshot(Seq(podWithIndex(1), podWithIndex(2))), + ExecutorPodsSnapshot(Seq(podWithIndex(1), podWithIndex(2), podWithIndex(3))))) + assert(receivedSnapshots1 === receivedSnapshots2) + } + + test("Even without sending events, initially receive an empty buffer.") { + val receivedInitialSnapshot = new AtomicReference[ExecutorPodsSnapshot](null) + eventQueueUnderTest.addSubscriber(1000) { + receivedInitialSnapshot.set + } + assert(receivedInitialSnapshot.get == null) + executeSubscriptionsExecutor.runUntilIdle() + assert(receivedInitialSnapshot.get === ExecutorPodsSnapshot()) + } + + test("Replacing the snapshot passes the new snapshot to subscribers.") { + val receivedSnapshots = mutable.Buffer.empty[ExecutorPodsSnapshot] + eventQueueUnderTest.addSubscriber(1000) { + receivedSnapshots += _ + } + eventQueueUnderTest.updatePod(podWithIndex(1)) + eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) + executeSubscriptionsExecutor.runUntilIdle() + assert(receivedSnapshots === Seq( + ExecutorPodsSnapshot(), + ExecutorPodsSnapshot(Seq(podWithIndex(1))))) + eventQueueUnderTest.replaceSnapshot(Seq(podWithIndex(2))) + eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) + executeSubscriptionsExecutor.runUntilIdle() + assert(receivedSnapshots === Seq( + ExecutorPodsSnapshot(), + ExecutorPodsSnapshot(Seq(podWithIndex(1))), + ExecutorPodsSnapshot(Seq(podWithIndex(2))))) + } + + private def pushPodWithIndex(index: Int): Unit = + eventQueueUnderTest.updatePod(podWithIndex(index)) + + private def podWithIndex(index: Int): Pod = + new PodBuilder() + .editOrNewMetadata() + .withName(s"pod-$index") + .addToLabels(SPARK_EXECUTOR_ID_LABEL, index.toString) + .endMetadata() + .editOrNewStatus() + .withPhase("running") + .endStatus() + .build() +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala index cabcc4be01374..eaf0928e6b197 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._ class ExecutorPodsWatchEventSourceSuite extends SparkFunSuite with BeforeAndAfter { @Mock - private var eventQueue: ExecutorPodsEventQueue = _ + private var eventQueue: ExecutorPodsSnapshotsStore = _ @Mock private var kubernetesClient: KubernetesClient = _ @@ -69,7 +69,7 @@ class ExecutorPodsWatchEventSourceSuite extends SparkFunSuite with BeforeAndAfte test("Watch events should be pushed to the queue.") { watch.getValue.eventReceived(Action.ADDED, runningExecutor(1)) watch.getValue.eventReceived(Action.MODIFIED, runningExecutor(2)) - verify(eventQueue).enqueue(runningExecutor(1)) - verify(eventQueue).enqueue(runningExecutor(2)) + verify(eventQueue).updatePod(runningExecutor(1)) + verify(eventQueue).updatePod(runningExecutor(2)) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 8f4db544d43ee..06fbd4e33b607 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -60,13 +60,13 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn private var taskScheduler: TaskSchedulerImpl = _ @Mock - private var eventQueue: ExecutorPodsEventQueue = _ + private var eventQueue: ExecutorPodsSnapshotsStore = _ @Mock private var podAllocator: ExecutorPodsAllocator = _ @Mock - private var lifecycleEventHandler: ExecutorPodsLifecycleEventHandler = _ + private var lifecycleEventHandler: ExecutorPodsLifecycleManager = _ @Mock private var watchEvents: ExecutorPodsWatchEventSource = _ From 3b85ab52b523dc182227b1ece517765903e64109 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 7 Jun 2018 18:36:36 -0700 Subject: [PATCH 36/47] Remove hanging comment --- .../spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index bfbd9d01df199..84671d1983951 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -54,8 +54,6 @@ private[spark] class ExecutorPodsAllocator( .withName(kubernetesDriverPodName) .get() - // Use sets of ids instead of counters to be able to handle duplicate events. - // Executor IDs that have been requested from Kubernetes but have not been detected in any // snapshot yet. Mapped to the timestamp when they were created. private val newlyCreatedExecutors = mutable.Map.empty[Long, Long] From edc982bb68d8da697a46def612fa270aa702001f Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 7 Jun 2018 18:37:43 -0700 Subject: [PATCH 37/47] Remove incorrect comment --- .../spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 84671d1983951..c32a9888baba0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -81,7 +81,6 @@ private[spark] class ExecutorPodsAllocator( // both the creation and deletion events. In either case, delete the missing pod // if possible, and mark such a pod to be rescheduled below. (newlyCreatedExecutors.keySet -- snapshot.executorPods.keySet).foreach { execId => - // Wait for 1 minute if (clock.getTimeMillis() - newlyCreatedExecutors(execId) > podCreationTimeout) { logWarning(s"Executor with id $execId was not detected in the Kubernetes" + " cluster after 1 minute despite the fact that a previous allocation attempt" + From e077c7e5d96016d39d3133431525c60054ea2374 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 7 Jun 2018 18:38:57 -0700 Subject: [PATCH 38/47] Fix log message --- .../spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index c32a9888baba0..895af10f234fe 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -83,9 +83,9 @@ private[spark] class ExecutorPodsAllocator( (newlyCreatedExecutors.keySet -- snapshot.executorPods.keySet).foreach { execId => if (clock.getTimeMillis() - newlyCreatedExecutors(execId) > podCreationTimeout) { logWarning(s"Executor with id $execId was not detected in the Kubernetes" + - " cluster after 1 minute despite the fact that a previous allocation attempt" + - " tried to create it. The executor may have been deleted but the application" + - " missed the deletion event.") + s" cluster after $podCreationTimeout milliseconds despite the fact that a" + + " previous allocation attempt tried to create it. The executor may have been" + + "deleted but the application missed the deletion event.") Utils.tryLogNonFatalError { kubernetesClient .pods() From a97fc5d5b87a8caf14ae923f89a7bc106c48d411 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 7 Jun 2018 18:40:33 -0700 Subject: [PATCH 39/47] Whitespace --- .../spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 895af10f234fe..ffbb93c241693 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -85,7 +85,7 @@ private[spark] class ExecutorPodsAllocator( logWarning(s"Executor with id $execId was not detected in the Kubernetes" + s" cluster after $podCreationTimeout milliseconds despite the fact that a" + " previous allocation attempt tried to create it. The executor may have been" + - "deleted but the application missed the deletion event.") + " deleted but the application missed the deletion event.") Utils.tryLogNonFatalError { kubernetesClient .pods() From bd7b0d3e5389300f1d0a3d2663cb73372cdc4dc3 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 7 Jun 2018 19:13:51 -0700 Subject: [PATCH 40/47] Clear all executors from the snapshot from newlyCreatedExecutors --- .../scheduler/cluster/k8s/ExecutorPodsAllocator.scala | 7 +------ .../scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala | 2 +- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index ffbb93c241693..a4670bfec4dc2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -67,12 +67,7 @@ private[spark] class ExecutorPodsAllocator( def setTotalExpectedExecutors(total: Int): Unit = totalExpectedExecutors.set(total) private def processSnapshot(applicationId: String, snapshot: ExecutorPodsSnapshot): Unit = { - snapshot.executorPods.filter { - case (_, PodPending(_)) | (_, PodUnknown(_)) => false - case _ => true - }.keys.foreach { - newlyCreatedExecutors -= _ - } + snapshot.executorPods.keys.foreach { newlyCreatedExecutors -= _ } // For all executors we've created against the API but have not seen in a snapshot // yet - check the current time. If the current time has exceeded some threshold, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index d6008824ad2b7..0c19f5946b75f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -131,7 +131,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) } - test("When an executor is requested but the API does not report it in 1 minute, retry" + + test("When an executor is requested but the API does not report it in a reasonable time, retry" + " requesting that executor.") { podsAllocatorUnderTest.setTotalExpectedExecutors(1) snapshotsStore.replaceSnapshot(Seq.empty[Pod]) From e9d7c8f779e2969cfa2ba159e1c7295cebddc27b Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 7 Jun 2018 19:30:26 -0700 Subject: [PATCH 41/47] Add a TODO for dynamic allocation --- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 1908ce0ce0d6a..aa32e13493588 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -99,6 +99,8 @@ private[spark] class KubernetesClusterSchedulerBackend( } override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { + // TODO when we support dynamic allocation, the pod allocator should be told to process the + // current snapshot in order to decrease/increase the number of executors accordingly. podAllocator.setTotalExpectedExecutors(requestedTotal) true } From 0fac4d5d9f754c3f10eb785933ae9ebb1210a728 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 8 Jun 2018 15:03:08 -0700 Subject: [PATCH 42/47] Address comments. Pass whole buffer of snapshots. Allows subscribers to choose how to process snapshots - for some operations it's important to process all snapshots but for other operations it's fine just to operate on the latest. --- .../cluster/k8s/ExecutorPodStates.scala | 10 +- .../cluster/k8s/ExecutorPodsAllocator.scala | 93 +++++++++--------- .../k8s/ExecutorPodsLifecycleManager.scala | 96 +++++++++++-------- .../k8s/ExecutorPodsSnapshotsStore.scala | 2 +- .../k8s/ExecutorPodsSnapshotsStoreImpl.scala | 4 +- ...erministicExecutorPodsSnapshotsStore.scala | 11 +-- .../k8s/ExecutorPodsSnapshotsStoreSuite.scala | 10 +- 7 files changed, 126 insertions(+), 100 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala index 59da53e3c3a3c..83daddf714489 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala @@ -24,12 +24,14 @@ sealed trait ExecutorPodState { case class PodRunning(pod: Pod) extends ExecutorPodState -case class PodSucceeded(pod: Pod) extends ExecutorPodState +case class PodPending(pod: Pod) extends ExecutorPodState -case class PodFailed(pod: Pod) extends ExecutorPodState +sealed trait FinalPodState extends ExecutorPodState -case class PodDeleted(pod: Pod) extends ExecutorPodState +case class PodSucceeded(pod: Pod) extends FinalPodState -case class PodPending(pod: Pod) extends ExecutorPodState +case class PodFailed(pod: Pod) extends FinalPodState + +case class PodDeleted(pod: Pod) extends FinalPodState case class PodUnknown(pod: Pod) extends ExecutorPodState diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index a4670bfec4dc2..c2adef8e889dd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -60,23 +60,22 @@ private[spark] class ExecutorPodsAllocator( def start(applicationId: String): Unit = { snapshotsStore.addSubscriber(podAllocationDelay) { - processSnapshot(applicationId, _) + onNewSnapshots(applicationId, _) } } def setTotalExpectedExecutors(total: Int): Unit = totalExpectedExecutors.set(total) - private def processSnapshot(applicationId: String, snapshot: ExecutorPodsSnapshot): Unit = { - snapshot.executorPods.keys.foreach { newlyCreatedExecutors -= _ } - + private def onNewSnapshots(applicationId: String, snapshots: Seq[ExecutorPodsSnapshot]): Unit = { + newlyCreatedExecutors --= snapshots.flatMap(_.executorPods.keys) // For all executors we've created against the API but have not seen in a snapshot // yet - check the current time. If the current time has exceeded some threshold, // assume that the pod was either never created (the API server never properly // handled the creation request), or the API server created the pod but we missed // both the creation and deletion events. In either case, delete the missing pod // if possible, and mark such a pod to be rescheduled below. - (newlyCreatedExecutors.keySet -- snapshot.executorPods.keySet).foreach { execId => - if (clock.getTimeMillis() - newlyCreatedExecutors(execId) > podCreationTimeout) { + newlyCreatedExecutors.foreach { case (execId, timeCreated) => + if (clock.getTimeMillis() - timeCreated > podCreationTimeout) { logWarning(s"Executor with id $execId was not detected in the Kubernetes" + s" cluster after $podCreationTimeout milliseconds despite the fact that a" + " previous allocation attempt tried to create it. The executor may have been" + @@ -91,44 +90,52 @@ private[spark] class ExecutorPodsAllocator( } } - val currentRunningExecutors = snapshot.executorPods.values.count { - case PodRunning(_) => true - case _ => false - } - val currentPendingExecutors = snapshot.executorPods.values.count { - case PodPending(_) => true - case _ => false - } - val currentTotalExpectedExecutors = totalExpectedExecutors.get - if (newlyCreatedExecutors.isEmpty - && currentPendingExecutors == 0 - && currentRunningExecutors < currentTotalExpectedExecutors) { - val numExecutorsToAllocate = math.min( - currentTotalExpectedExecutors - currentRunningExecutors, podAllocationSize) - logInfo(s"Going to request $numExecutorsToAllocate executors from Kubernetes.") - for ( _ <- 0 until numExecutorsToAllocate) { - val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet() - val executorConf = KubernetesConf.createExecutorConf( - conf, - newExecutorId.toString, - applicationId, - driverPod) - val executorPod = executorBuilder.buildFromFeatures(executorConf) - val podWithAttachedContainer = new PodBuilder(executorPod.pod) - .editOrNewSpec() - .addToContainers(executorPod.container) - .endSpec() - .build() - kubernetesClient.pods().create(podWithAttachedContainer) - newlyCreatedExecutors(newExecutorId) = clock.getTimeMillis() + if (snapshots.nonEmpty) { + // Only need to examine the cluster as of the latest snapshot, the "current" state, to see if + // we need to allocate more executors or not. + val latestSnapshot = snapshots.last + val currentRunningExecutors = latestSnapshot.executorPods.values.count { + case PodRunning(_) => true + case _ => false + } + val currentPendingExecutors = latestSnapshot.executorPods.values.count { + case PodPending(_) => true + case _ => false + } + val currentTotalExpectedExecutors = totalExpectedExecutors.get + if (newlyCreatedExecutors.isEmpty + && currentPendingExecutors == 0 + && currentRunningExecutors < currentTotalExpectedExecutors) { + val numExecutorsToAllocate = math.min( + currentTotalExpectedExecutors - currentRunningExecutors, podAllocationSize) + logInfo(s"Going to request $numExecutorsToAllocate executors from Kubernetes.") + for ( _ <- 0 until numExecutorsToAllocate) { + val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet() + val executorConf = KubernetesConf.createExecutorConf( + conf, + newExecutorId.toString, + applicationId, + driverPod) + val executorPod = executorBuilder.buildFromFeatures(executorConf) + val podWithAttachedContainer = new PodBuilder(executorPod.pod) + .editOrNewSpec() + .addToContainers(executorPod.container) + .endSpec() + .build() + kubernetesClient.pods().create(podWithAttachedContainer) + newlyCreatedExecutors(newExecutorId) = clock.getTimeMillis() + } + } else if (currentRunningExecutors >= currentTotalExpectedExecutors) { + // TODO handle edge cases if we end up with more running executors than expected. + logDebug("Current number of running executors is equal to the number of requested" + + " executors. Not scaling up further.") + } else if (newlyCreatedExecutors.nonEmpty || currentPendingExecutors != 0) { + logDebug(s"Still waiting for ${newlyCreatedExecutors.size + currentPendingExecutors}" + + s" executors to begin running before requesting for more executors. # of executors in" + + s" pending status in the cluster: $currentPendingExecutors. # of executors that we have" + + s" created but we have not observed as being present in the cluster yet:" + + s" ${newlyCreatedExecutors.size}.") } - } else if (currentRunningExecutors >= currentTotalExpectedExecutors) { - // TODO handle edge cases if we end up with more running executors than expected. - logDebug("Current number of running executors is equal to the number of requested" + - " executors. Not scaling up further.") - } else if (newlyCreatedExecutors.nonEmpty || currentPendingExecutors != 0) { - logDebug(s"Still waiting for ${newlyCreatedExecutors.size + currentPendingExecutors}" + - s" executors to begin running before requesting for more executors.") } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala index bc8205972f8c7..09477a8886421 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala @@ -44,45 +44,63 @@ private[spark] class ExecutorPodsLifecycleManager( def start(schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { snapshotsStore.addSubscriber(eventProcessingInterval) { - onNextSnapshot(schedulerBackend, _) + onNewSnapshots(schedulerBackend, _) } } - private def onNextSnapshot( + private def onNewSnapshots( schedulerBackend: KubernetesClusterSchedulerBackend, - snapshot: ExecutorPodsSnapshot): Unit = { + snapshots: Seq[ExecutorPodsSnapshot]): Unit = { val execIdsRemovedInThisRound = mutable.HashSet.empty[Long] - snapshot.executorPods.foreach { case (execId, state) => - state match { - case PodDeleted(pod) => - removeExecutorFromSpark(schedulerBackend, pod, execId) - execIdsRemovedInThisRound += execId - case errorOrSucceeded @ (PodFailed(_) | PodSucceeded(_)) => - removeExecutorFromK8s(errorOrSucceeded.pod) - removeExecutorFromSpark(schedulerBackend, errorOrSucceeded.pod, execId) - execIdsRemovedInThisRound += execId - case _ => + snapshots.foreach { snapshot => + snapshot.executorPods.foreach { case (execId, state) => + state match { + case deleted@PodDeleted(pod) => + removeExecutorFromSpark(schedulerBackend, deleted, execId) + execIdsRemovedInThisRound += execId + case failed@PodFailed(pod) => + onFinalNonDeletedState(failed, execId, schedulerBackend, execIdsRemovedInThisRound) + case succeeded@PodSucceeded(pod) => + onFinalNonDeletedState(succeeded, execId, schedulerBackend, execIdsRemovedInThisRound) + case _ => + } } } // Reconcile the case where Spark claims to know about an executor but the corresponding pod // is missing from the cluster. This would occur if we miss a deletion event and the pod - // transitions immediately from running io absent. - (schedulerBackend.getExecutorIds().map(_.toLong).toSet - -- snapshot.executorPods.keySet - -- execIdsRemovedInThisRound).foreach { missingExecutorId => - if (removedExecutorsCache.getIfPresent(missingExecutorId) == null) { - val exitReason = ExecutorExited( - UNKNOWN_EXIT_CODE, - exitCausedByApp = false, - s"The executor with ID $missingExecutorId was not found in the cluster but we didn't" + - s" get a reason why. Marking the executor as failed. The executor may have been" + - s" deleted but the driver missed the deletion event.") - schedulerBackend.doRemoveExecutor(missingExecutorId.toString, exitReason) + // transitions immediately from running io absent. We only need to check against the latest + // snapshot for this, and we don't do this for executors in the deleted executors cache or + // that we just removed in this round. + if (snapshots.nonEmpty) { + val latestSnapshot = snapshots.last + (schedulerBackend.getExecutorIds().map(_.toLong).toSet + -- latestSnapshot.executorPods.keySet + -- execIdsRemovedInThisRound).foreach { missingExecutorId => + if (removedExecutorsCache.getIfPresent(missingExecutorId) == null) { + val exitReason = ExecutorExited( + UNKNOWN_EXIT_CODE, + exitCausedByApp = false, + s"The executor with ID $missingExecutorId was not found in the cluster but we didn't" + + s" get a reason why. Marking the executor as failed. The executor may have been" + + s" deleted but the driver missed the deletion event.") + schedulerBackend.doRemoveExecutor(missingExecutorId.toString, exitReason) + execIdsRemovedInThisRound += missingExecutorId + } } } } + private def onFinalNonDeletedState( + podState: FinalPodState, + execId: Long, + schedulerBackend: KubernetesClusterSchedulerBackend, + execIdsRemovedInRound: mutable.Set[Long]): Unit = { + removeExecutorFromK8s(podState.pod) + removeExecutorFromSpark(schedulerBackend, podState, execId) + execIdsRemovedInRound += execId + } + private def removeExecutorFromK8s(updatedPod: Pod): Unit = { // If deletion failed on a previous try, we can try again if resync informs us the pod // is still around. @@ -98,27 +116,29 @@ private[spark] class ExecutorPodsLifecycleManager( private def removeExecutorFromSpark( schedulerBackend: KubernetesClusterSchedulerBackend, - pod: Pod, + podState: FinalPodState, execId: Long): Unit = { if (removedExecutorsCache.getIfPresent(execId) == null) { removedExecutorsCache.put(execId, execId) - val exitReason = findExitReason(pod, execId) + val exitReason = findExitReason(podState, execId) schedulerBackend.doRemoveExecutor(execId.toString, exitReason) } } - private def findExitReason(pod: Pod, execId: Long): ExecutorExited = { - val exitCode = findExitCode(pod) - val (exitCausedByApp, exitMessage) = if (isDeleted(pod)) { - (false, s"The executor with id $execId was deleted by a user or the framework.") - } else { - val msg = exitReasonMessage(pod, execId, exitCode) - (true, msg) + private def findExitReason(podState: FinalPodState, execId: Long): ExecutorExited = { + val exitCode = findExitCode(podState) + val (exitCausedByApp, exitMessage) = podState match { + case PodDeleted(_) => + (false, s"The executor with id $execId was deleted by a user or the framework.") + case _ => + val msg = exitReasonMessage(podState, execId, exitCode) + (true, msg) } ExecutorExited(exitCode, exitCausedByApp, exitMessage) } - private def exitReasonMessage(pod: Pod, execId: Long, exitCode: Int) = { + private def exitReasonMessage(podState: FinalPodState, execId: Long, exitCode: Int) = { + val pod = podState.pod s""" |The executor with id $execId exited with exit code $exitCode. |The API gave the following brief reason: ${pod.getStatus.getReason} @@ -129,10 +149,8 @@ private[spark] class ExecutorPodsLifecycleManager( """.stripMargin } - private def isDeleted(pod: Pod): Boolean = pod.getMetadata.getDeletionTimestamp != null - - private def findExitCode(pod: Pod): Int = { - pod.getStatus.getContainerStatuses.asScala.find { containerStatus => + private def findExitCode(podState: FinalPodState): Int = { + podState.pod.getStatus.getContainerStatuses.asScala.find { containerStatus => containerStatus.getState.getTerminated != null }.map { terminatedContainer => terminatedContainer.getState.getTerminated.getExitCode.toInt diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala index 37c1c7f6b6213..dd264332cf9e8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala @@ -22,7 +22,7 @@ private[spark] trait ExecutorPodsSnapshotsStore { def addSubscriber (processBatchIntervalMillis: Long) - (onNextSnapshot: ExecutorPodsSnapshot => Unit) + (onNewSnapshots: Seq[ExecutorPodsSnapshot] => Unit) def stop(): Unit diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala index 5abdd4f18f1e5..f3aec429153f0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala @@ -45,7 +45,7 @@ private[spark] class ExecutorPodsSnapshotsStoreImpl( override def addSubscriber( processBatchIntervalMillis: Long) - (subscriber: ExecutorPodsSnapshot => Unit): Unit = { + (onNewSnapshots: Seq[ExecutorPodsSnapshot] => Unit): Unit = { observedDisposables += snapshotsObservable // Group events in the time window given by the caller. These buffers are then sent // to the caller's lambda at the given interval, with the pod updates that occurred @@ -65,7 +65,7 @@ private[spark] class ExecutorPodsSnapshotsStoreImpl( .observeOn(Schedulers.from(executeSubscriptionsExecutor)) .subscribe(toReactivexConsumer { snapshots: java.util.List[ExecutorPodsSnapshot] => Utils.tryLogNonFatalError { - snapshots.asScala.foreach(subscriber) + onNewSnapshots(snapshots.asScala) } }) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala index 3c5b62441b210..f7721e6fd6388 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala @@ -22,24 +22,23 @@ import scala.collection.mutable class DeterministicExecutorPodsSnapshotsStore extends ExecutorPodsSnapshotsStore { private val snapshotsBuffer = mutable.Buffer.empty[ExecutorPodsSnapshot] - private val subscribers = mutable.Buffer.empty[ExecutorPodsSnapshot => Unit] + private val subscribers = mutable.Buffer.empty[Seq[ExecutorPodsSnapshot] => Unit] private var currentSnapshot = ExecutorPodsSnapshot() override def addSubscriber (processBatchIntervalMillis: Long) - (subscriber: ExecutorPodsSnapshot => Unit): Unit = { - subscribers += subscriber + (onNewSnapshots: Seq[ExecutorPodsSnapshot] => Unit): Unit = { + subscribers += onNewSnapshots } override def stop(): Unit = {} def notifySubscribers(): Unit = { - subscribers.foreach { subscriber => - snapshotsBuffer.foreach(subscriber) - } + subscribers.foreach(_(snapshotsBuffer)) snapshotsBuffer.clear() } + override def updatePod(updatedPod: Pod): Unit = { currentSnapshot = currentSnapshot.withUpdate(updatedPod) snapshotsBuffer += currentSnapshot diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala index 708af04ed0243..f7d2676bbd030 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala @@ -45,10 +45,10 @@ class ExecutorPodsSnapshotsStoreSuite extends SparkFunSuite with BeforeAndAfter val receivedSnapshots1 = mutable.Buffer.empty[ExecutorPodsSnapshot] val receivedSnapshots2 = mutable.Buffer.empty[ExecutorPodsSnapshot] eventQueueUnderTest.addSubscriber(1000) { - receivedSnapshots1 += _ + receivedSnapshots1 ++= _ } eventQueueUnderTest.addSubscriber(2000) { - receivedSnapshots2 += _ + receivedSnapshots2 ++= _ } pushPodWithIndex(1) @@ -101,19 +101,19 @@ class ExecutorPodsSnapshotsStoreSuite extends SparkFunSuite with BeforeAndAfter } test("Even without sending events, initially receive an empty buffer.") { - val receivedInitialSnapshot = new AtomicReference[ExecutorPodsSnapshot](null) + val receivedInitialSnapshot = new AtomicReference[Seq[ExecutorPodsSnapshot]](null) eventQueueUnderTest.addSubscriber(1000) { receivedInitialSnapshot.set } assert(receivedInitialSnapshot.get == null) executeSubscriptionsExecutor.runUntilIdle() - assert(receivedInitialSnapshot.get === ExecutorPodsSnapshot()) + assert(receivedInitialSnapshot.get === Seq(ExecutorPodsSnapshot())) } test("Replacing the snapshot passes the new snapshot to subscribers.") { val receivedSnapshots = mutable.Buffer.empty[ExecutorPodsSnapshot] eventQueueUnderTest.addSubscriber(1000) { - receivedSnapshots += _ + receivedSnapshots ++= _ } eventQueueUnderTest.updatePod(podWithIndex(1)) eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) From c1b8431524474ae710e2733836d86e1ee94df02f Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 8 Jun 2018 15:54:35 -0700 Subject: [PATCH 43/47] Rename classes --- ...e.scala => ExecutorPodsPollingSnapshotSource.scala} | 2 +- ...rce.scala => ExecutorPodsWatchSnapshotSource.scala} | 2 +- .../cluster/k8s/KubernetesClusterManager.scala | 4 ++-- .../k8s/KubernetesClusterSchedulerBackend.scala | 4 ++-- ...la => ExecutorPodsPollingSnapshotSourceSuite.scala} | 6 +++--- ...cala => ExecutorPodsWatchSnapshotSourceSuite.scala} | 10 +++++----- .../k8s/KubernetesClusterSchedulerBackendSuite.scala | 4 ++-- 7 files changed, 16 insertions(+), 16 deletions(-) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/{ExecutorPodsPollingEventSource.scala => ExecutorPodsPollingSnapshotSource.scala} (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/{ExecutorPodsWatchEventSource.scala => ExecutorPodsWatchSnapshotSource.scala} (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/{ExecutorPodsPollingEventSourceSuite.scala => ExecutorPodsPollingSnapshotSourceSuite.scala} (92%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/{ExecutorPodsWatchEventSourceSuite.scala => ExecutorPodsWatchSnapshotSourceSuite.scala} (87%) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala index 6a664f4edf612..028aa01ac9324 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala @@ -26,7 +26,7 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.util.ThreadUtils -private[spark] class ExecutorPodsPollingEventSource( +private[spark] class ExecutorPodsPollingSnapshotSource( conf: SparkConf, kubernetesClient: KubernetesClient, snapshotsStore: ExecutorPodsSnapshotsStore, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala index b82ad338b9132..4dfea8fad94f6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala @@ -26,7 +26,7 @@ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -private[spark] class ExecutorPodsWatchEventSource( +private[spark] class ExecutorPodsWatchSnapshotSource( snapshotsStore: ExecutorPodsSnapshotsStore, kubernetesClient: KubernetesClient) extends Logging { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 0ae7e274c0e36..76de834f12205 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -80,13 +80,13 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val executorPodsAllocator = new ExecutorPodsAllocator( sc.conf, new KubernetesExecutorBuilder(), kubernetesClient, snapshotsStore, new SystemClock()) - val podsWatchEventSource = new ExecutorPodsWatchEventSource( + val podsWatchEventSource = new ExecutorPodsWatchSnapshotSource( snapshotsStore, kubernetesClient) val eventsPollingExecutor = ThreadUtils.newDaemonSingleThreadScheduledExecutor( "kubernetes-executor-pod-polling-sync") - val podsPollingEventSource = new ExecutorPodsPollingEventSource( + val podsPollingEventSource = new ExecutorPodsPollingSnapshotSource( sc.conf, kubernetesClient, snapshotsStore, eventsPollingExecutor) new KubernetesClusterSchedulerBackend( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index aa32e13493588..fa6dc2c479bbf 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -35,8 +35,8 @@ private[spark] class KubernetesClusterSchedulerBackend( snapshotsStore: ExecutorPodsSnapshotsStore, podAllocator: ExecutorPodsAllocator, lifecycleEventHandler: ExecutorPodsLifecycleManager, - watchEvents: ExecutorPodsWatchEventSource, - pollEvents: ExecutorPodsPollingEventSource) + watchEvents: ExecutorPodsWatchSnapshotSource, + pollEvents: ExecutorPodsPollingSnapshotSource) extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSourceSuite.scala similarity index 92% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSourceSuite.scala index 50a2d9b0aacf4..1b26d6af296a5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingEventSourceSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSourceSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._ -class ExecutorPodsPollingEventSourceSuite extends SparkFunSuite with BeforeAndAfter { +class ExecutorPodsPollingSnapshotSourceSuite extends SparkFunSuite with BeforeAndAfter { private val sparkConf = new SparkConf @@ -53,12 +53,12 @@ class ExecutorPodsPollingEventSourceSuite extends SparkFunSuite with BeforeAndAf private var eventQueue: ExecutorPodsSnapshotsStore = _ private var pollingExecutor: DeterministicScheduler = _ - private var pollingSourceUnderTest: ExecutorPodsPollingEventSource = _ + private var pollingSourceUnderTest: ExecutorPodsPollingSnapshotSource = _ before { MockitoAnnotations.initMocks(this) pollingExecutor = new DeterministicScheduler() - pollingSourceUnderTest = new ExecutorPodsPollingEventSource( + pollingSourceUnderTest = new ExecutorPodsPollingSnapshotSource( sparkConf, kubernetesClient, eventQueue, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSourceSuite.scala similarity index 87% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSourceSuite.scala index eaf0928e6b197..ac1968b4ff810 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchEventSourceSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSourceSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._ -class ExecutorPodsWatchEventSourceSuite extends SparkFunSuite with BeforeAndAfter { +class ExecutorPodsWatchSnapshotSourceSuite extends SparkFunSuite with BeforeAndAfter { @Mock private var eventQueue: ExecutorPodsSnapshotsStore = _ @@ -50,7 +50,7 @@ class ExecutorPodsWatchEventSourceSuite extends SparkFunSuite with BeforeAndAfte private var watch: ArgumentCaptor[Watcher[Pod]] = _ - private var watchEventSourceUnderTest: ExecutorPodsWatchEventSource = _ + private var watchSourceUnderTest: ExecutorPodsWatchSnapshotSource = _ before { MockitoAnnotations.initMocks(this) @@ -61,12 +61,12 @@ class ExecutorPodsWatchEventSourceSuite extends SparkFunSuite with BeforeAndAfte when(appIdLabeledPods.withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) .thenReturn(executorRoleLabeledPods) when(executorRoleLabeledPods.watch(watch.capture())).thenReturn(watchConnection) - watchEventSourceUnderTest = new ExecutorPodsWatchEventSource( + watchSourceUnderTest = new ExecutorPodsWatchSnapshotSource( eventQueue, kubernetesClient) - watchEventSourceUnderTest.start(TEST_SPARK_APP_ID) + watchSourceUnderTest.start(TEST_SPARK_APP_ID) } - test("Watch events should be pushed to the queue.") { + test("Watch events should be pushed to the snapshots store as snapshot updates.") { watch.getValue.eventReceived(Action.ADDED, runningExecutor(1)) watch.getValue.eventReceived(Action.MODIFIED, runningExecutor(2)) verify(eventQueue).updatePod(runningExecutor(1)) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 06fbd4e33b607..52e7a12dbaf06 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -69,10 +69,10 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn private var lifecycleEventHandler: ExecutorPodsLifecycleManager = _ @Mock - private var watchEvents: ExecutorPodsWatchEventSource = _ + private var watchEvents: ExecutorPodsWatchSnapshotSource = _ @Mock - private var pollEvents: ExecutorPodsPollingEventSource = _ + private var pollEvents: ExecutorPodsPollingSnapshotSource = _ private var driverEndpoint: ArgumentCaptor[RpcEndpoint] = _ private var schedulerBackendUnderTest: KubernetesClusterSchedulerBackend = _ From 108181d8190b7bc15eb80caeceff2ce8103284a4 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 12 Jun 2018 12:08:44 -0700 Subject: [PATCH 44/47] Don't use RxJava --- pom.xml | 5 -- resource-managers/kubernetes/core/pom.xml | 5 -- .../k8s/ExecutorPodsSnapshotsStoreImpl.scala | 77 +++++++++---------- .../k8s/KubernetesClusterManager.scala | 5 +- .../k8s/ExecutorPodsSnapshotsStoreSuite.scala | 27 ++----- 5 files changed, 41 insertions(+), 78 deletions(-) diff --git a/pom.xml b/pom.xml index 40e1b198b2584..4b4e6c13ea8fd 100644 --- a/pom.xml +++ b/pom.xml @@ -748,11 +748,6 @@ jline 2.12.1 - - io.reactivex.rxjava2 - rxjava - 2.1.13 - org.scalatest scalatest_${scala.binary.version} diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 55c15c952c938..a6dd47a6b7d95 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -83,11 +83,6 @@ 3.8.1 - - io.reactivex.rxjava2 - rxjava - - org.mockito mockito-core diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala index f3aec429153f0..59548c056d844 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala @@ -16,29 +16,22 @@ */ package org.apache.spark.scheduler.cluster.k8s -import java.util.concurrent.{ExecutorService, ScheduledExecutorService, TimeUnit} +import java.util.concurrent._ -import com.google.common.collect.Lists import io.fabric8.kubernetes.api.model.Pod -import io.reactivex.disposables.Disposable -import io.reactivex.functions.Consumer -import io.reactivex.schedulers.Schedulers -import io.reactivex.subjects.PublishSubject import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.util.{ThreadUtils, Utils} -private[spark] class ExecutorPodsSnapshotsStoreImpl( - bufferSnapshotsExecutor: ScheduledExecutorService, - executeSubscriptionsExecutor: ExecutorService) +private[spark] class ExecutorPodsSnapshotsStoreImpl(subscribersExecutor: ScheduledExecutorService) extends ExecutorPodsSnapshotsStore { private val SNAPSHOT_LOCK = new Object() - private val snapshotsObservable = PublishSubject.create[ExecutorPodsSnapshot]() - private val observedDisposables = mutable.Buffer.empty[Disposable] + private val subscribers = mutable.Buffer.empty[SnapshotsSubscriber] + private val pollingTasks = mutable.Buffer.empty[Future[_]] @GuardedBy("SNAPSHOT_LOCK") private var currentSnapshot = ExecutorPodsSnapshot() @@ -46,50 +39,50 @@ private[spark] class ExecutorPodsSnapshotsStoreImpl( override def addSubscriber( processBatchIntervalMillis: Long) (onNewSnapshots: Seq[ExecutorPodsSnapshot] => Unit): Unit = { - observedDisposables += snapshotsObservable - // Group events in the time window given by the caller. These buffers are then sent - // to the caller's lambda at the given interval, with the pod updates that occurred - // in that given interval. - .buffer( - processBatchIntervalMillis, - TimeUnit.MILLISECONDS, - // For testing - specifically use the given scheduled executor service to trigger - // buffer boundaries. Allows us to inject a deterministic scheduler here. - Schedulers.from(bufferSnapshotsExecutor)) - // Trigger an event cycle immediately. Not strictly required to be fully correct, but - // in particular the pod allocator should try to request executors immediately instead - // of waiting for one pod allocation delay. - .startWith(Lists.newArrayList(ExecutorPodsSnapshot())) - // Force all triggered events - both the initial event above and the buffered ones in - // the following time windows - to execute asynchronously to this call's thread. - .observeOn(Schedulers.from(executeSubscriptionsExecutor)) - .subscribe(toReactivexConsumer { snapshots: java.util.List[ExecutorPodsSnapshot] => - Utils.tryLogNonFatalError { - onNewSnapshots(snapshots.asScala) - } - }) + val newSubscriber = SnapshotsSubscriber( + new LinkedBlockingQueue[ExecutorPodsSnapshot](), onNewSnapshots) + subscribers += newSubscriber + pollingTasks += subscribersExecutor.scheduleWithFixedDelay( + toRunnable(() => callSubscriber(newSubscriber)), + 0L, + processBatchIntervalMillis, + TimeUnit.MILLISECONDS) } override def stop(): Unit = { - observedDisposables.foreach(_.dispose()) - snapshotsObservable.onComplete() - ThreadUtils.shutdown(bufferSnapshotsExecutor) - ThreadUtils.shutdown(executeSubscriptionsExecutor) + pollingTasks.foreach(_.cancel(true)) + ThreadUtils.shutdown(subscribersExecutor) } override def updatePod(updatedPod: Pod): Unit = SNAPSHOT_LOCK.synchronized { currentSnapshot = currentSnapshot.withUpdate(updatedPod) - snapshotsObservable.onNext(currentSnapshot) + addCurrentSnapshotToSubscribers() } override def replaceSnapshot(newSnapshot: Seq[Pod]): Unit = SNAPSHOT_LOCK.synchronized { currentSnapshot = ExecutorPodsSnapshot(newSnapshot) - snapshotsObservable.onNext(currentSnapshot) + addCurrentSnapshotToSubscribers() } - private def toReactivexConsumer[T](consumer: T => Unit): Consumer[T] = { - new Consumer[T] { - override def accept(item: T): Unit = consumer(item) + private def addCurrentSnapshotToSubscribers(): Unit = { + subscribers.foreach { subscriber => + subscriber.snapshotsBuffer.add(currentSnapshot) } } + + private def callSubscriber(subscriber: SnapshotsSubscriber): Unit = { + Utils.tryLogNonFatalError { + val currentSnapshots = mutable.Buffer.empty[ExecutorPodsSnapshot].asJava + subscriber.snapshotsBuffer.drainTo(currentSnapshots) + subscriber.onNewSnapshots(currentSnapshots.asScala) + } + } + + private def toRunnable[T](runnable: () => Unit): Runnable = new Runnable { + override def run(): Unit = runnable() + } + + private case class SnapshotsSubscriber( + snapshotsBuffer: BlockingQueue[ExecutorPodsSnapshot], + onNewSnapshots: (Seq[ExecutorPodsSnapshot] => Unit)) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 76de834f12205..cd10890fbbd1a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -63,10 +63,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val bufferSnapshotsExecutor = ThreadUtils .newDaemonSingleThreadScheduledExecutor("kubernetes-executor-snapshots-buffer") - val executeSubscribersExecutor = ThreadUtils - .newDaemonCachedThreadPool("kubernetes-executor-snapshots-handlers") - val snapshotsStore = new ExecutorPodsSnapshotsStoreImpl( - bufferSnapshotsExecutor, executeSubscribersExecutor) + val snapshotsStore = new ExecutorPodsSnapshotsStoreImpl(bufferSnapshotsExecutor) val removedExecutorsCache = CacheBuilder.newBuilder() .expireAfterWrite(3, TimeUnit.MINUTES) .build[java.lang.Long, java.lang.Long]() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala index f7d2676bbd030..87e0b03607f70 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala @@ -30,15 +30,11 @@ import org.apache.spark.deploy.k8s.Constants._ class ExecutorPodsSnapshotsStoreSuite extends SparkFunSuite with BeforeAndAfter { private var eventBufferScheduler: DeterministicScheduler = _ - private var executeSubscriptionsExecutor: DeterministicScheduler = _ private var eventQueueUnderTest: ExecutorPodsSnapshotsStoreImpl = _ before { eventBufferScheduler = new DeterministicScheduler() - executeSubscriptionsExecutor = new DeterministicScheduler() - eventQueueUnderTest = new ExecutorPodsSnapshotsStoreImpl( - eventBufferScheduler, - executeSubscriptionsExecutor) + eventQueueUnderTest = new ExecutorPodsSnapshotsStoreImpl(eventBufferScheduler) } test("Subscribers get notified of events periodically.") { @@ -50,6 +46,7 @@ class ExecutorPodsSnapshotsStoreSuite extends SparkFunSuite with BeforeAndAfter eventQueueUnderTest.addSubscriber(2000) { receivedSnapshots2 ++= _ } + eventBufferScheduler.runUntilIdle() pushPodWithIndex(1) assert(receivedSnapshots1.isEmpty) @@ -58,42 +55,32 @@ class ExecutorPodsSnapshotsStoreSuite extends SparkFunSuite with BeforeAndAfter // processing task on the subscription executor... eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) // ... then actually execute the subscribers. - executeSubscriptionsExecutor.runUntilIdle() assert(receivedSnapshots1 === Seq( - ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))))) - assert(receivedSnapshots2 === Seq(ExecutorPodsSnapshot())) + assert(receivedSnapshots2.isEmpty) eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) - executeSubscriptionsExecutor.runUntilIdle() // Don't repeat snapshots assert(receivedSnapshots1 === Seq( - ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))))) assert(receivedSnapshots2 === Seq( - ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1)))) ) pushPodWithIndex(2) pushPodWithIndex(3) eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) - executeSubscriptionsExecutor.runUntilIdle() assert(receivedSnapshots1 === Seq( - ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))), ExecutorPodsSnapshot(Seq(podWithIndex(1), podWithIndex(2))), ExecutorPodsSnapshot(Seq(podWithIndex(1), podWithIndex(2), podWithIndex(3))))) assert(receivedSnapshots2 === Seq( - ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))))) eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) - executeSubscriptionsExecutor.runUntilIdle() assert(receivedSnapshots1 === Seq( - ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))), ExecutorPodsSnapshot(Seq(podWithIndex(1), podWithIndex(2))), ExecutorPodsSnapshot(Seq(podWithIndex(1), podWithIndex(2), podWithIndex(3))))) @@ -106,8 +93,8 @@ class ExecutorPodsSnapshotsStoreSuite extends SparkFunSuite with BeforeAndAfter receivedInitialSnapshot.set } assert(receivedInitialSnapshot.get == null) - executeSubscriptionsExecutor.runUntilIdle() - assert(receivedInitialSnapshot.get === Seq(ExecutorPodsSnapshot())) + eventBufferScheduler.runUntilIdle() + assert(receivedInitialSnapshot.get.isEmpty) } test("Replacing the snapshot passes the new snapshot to subscribers.") { @@ -117,15 +104,11 @@ class ExecutorPodsSnapshotsStoreSuite extends SparkFunSuite with BeforeAndAfter } eventQueueUnderTest.updatePod(podWithIndex(1)) eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) - executeSubscriptionsExecutor.runUntilIdle() assert(receivedSnapshots === Seq( - ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))))) eventQueueUnderTest.replaceSnapshot(Seq(podWithIndex(2))) eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) - executeSubscriptionsExecutor.runUntilIdle() assert(receivedSnapshots === Seq( - ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))), ExecutorPodsSnapshot(Seq(podWithIndex(2))))) } From 9e0b75845c3f729a7bca92033b5543cebc237321 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 12 Jun 2018 12:12:51 -0700 Subject: [PATCH 45/47] Update manifest --- dev/deps/spark-deps-hadoop-2.6 | 2 -- dev/deps/spark-deps-hadoop-2.7 | 2 -- dev/deps/spark-deps-hadoop-3.1 | 2 -- 3 files changed, 6 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index ad26444921b87..723180a14febb 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -172,8 +172,6 @@ parquet-jackson-1.10.0.jar protobuf-java-2.5.0.jar py4j-0.10.7.jar pyrolite-4.13.jar -reactive-streams-1.0.2.jar -rxjava-2.1.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 9d34588e1417d..ea08a001a1c9b 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -173,8 +173,6 @@ parquet-jackson-1.10.0.jar protobuf-java-2.5.0.jar py4j-0.10.7.jar pyrolite-4.13.jar -reactive-streams-1.0.2.jar -rxjava-2.1.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index f5b2c381151ab..da874026d7d10 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -192,8 +192,6 @@ protobuf-java-2.5.0.jar py4j-0.10.7.jar pyrolite-4.13.jar re2j-1.1.jar -reactive-streams-1.0.2.jar -rxjava-2.1.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar From 8b0a211f4c5046065b0e2abcac0ee71119ff6bd6 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 12 Jun 2018 12:43:31 -0700 Subject: [PATCH 46/47] Remove extra parens --- .../scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala index 59548c056d844..8776a525a084a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala @@ -84,5 +84,5 @@ private[spark] class ExecutorPodsSnapshotsStoreImpl(subscribersExecutor: Schedul private case class SnapshotsSubscriber( snapshotsBuffer: BlockingQueue[ExecutorPodsSnapshot], - onNewSnapshots: (Seq[ExecutorPodsSnapshot] => Unit)) + onNewSnapshots: Seq[ExecutorPodsSnapshot] => Unit) } From 1a99dceeb9dfbfc58e26885c290461cbf37a5428 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 14 Jun 2018 13:58:06 -0700 Subject: [PATCH 47/47] Address comments. Make subscriber thread pool instead of single thread --- .../org/apache/spark/util/ThreadUtils.scala | 16 ++++++++++++ .../cluster/k8s/ExecutorPodsAllocator.scala | 10 ++++++- .../k8s/ExecutorPodsLifecycleManager.scala | 26 ++++++++++++++----- .../ExecutorPodsPollingSnapshotSource.scala | 5 +++- .../k8s/ExecutorPodsSnapshotsStoreImpl.scala | 25 ++++++++++++++++++ .../k8s/ExecutorPodsWatchSnapshotSource.scala | 4 +++ .../k8s/KubernetesClusterManager.scala | 7 ++--- .../k8s/ExecutorPodsSnapshotsStoreSuite.scala | 20 +++++++++----- 8 files changed, 95 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala index e31a76125d341..0f08a2b0ad895 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala @@ -102,6 +102,22 @@ private[spark] object ThreadUtils { executor } + /** + * Wrapper over ScheduledThreadPoolExecutor. + */ + def newDaemonThreadPoolScheduledExecutor(threadNamePrefix: String, numThreads: Int) + : ScheduledExecutorService = { + val threadFactory = new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(s"$threadNamePrefix-%d") + .build() + val executor = new ScheduledThreadPoolExecutor(numThreads, threadFactory) + // By default, a cancelled task is not automatically removed from the work queue until its delay + // elapses. We have to enable it manually. + executor.setRemoveOnCancelPolicy(true) + executor + } + /** * Run a piece of code in a new thread and return the result. Exception in the new thread is * thrown in the caller thread with an adjusted stack trace that removes references to this diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index c2adef8e889dd..5a143ad3600fd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -75,7 +75,8 @@ private[spark] class ExecutorPodsAllocator( // both the creation and deletion events. In either case, delete the missing pod // if possible, and mark such a pod to be rescheduled below. newlyCreatedExecutors.foreach { case (execId, timeCreated) => - if (clock.getTimeMillis() - timeCreated > podCreationTimeout) { + val currentTime = clock.getTimeMillis() + if (currentTime - timeCreated > podCreationTimeout) { logWarning(s"Executor with id $execId was not detected in the Kubernetes" + s" cluster after $podCreationTimeout milliseconds despite the fact that a" + " previous allocation attempt tried to create it. The executor may have been" + @@ -87,6 +88,9 @@ private[spark] class ExecutorPodsAllocator( .delete() } newlyCreatedExecutors -= execId + } else { + logDebug(s"Executor with id $execId was not found in the Kubernetes cluster since it" + + s" was created ${currentTime - timeCreated} milliseconds ago.") } } @@ -103,6 +107,9 @@ private[spark] class ExecutorPodsAllocator( case _ => false } val currentTotalExpectedExecutors = totalExpectedExecutors.get + logDebug(s"Currently have $currentRunningExecutors running executors and" + + s" $currentPendingExecutors pending executors. $newlyCreatedExecutors executors" + + s" have been requested but are pending appearance in the cluster.") if (newlyCreatedExecutors.isEmpty && currentPendingExecutors == 0 && currentRunningExecutors < currentTotalExpectedExecutors) { @@ -124,6 +131,7 @@ private[spark] class ExecutorPodsAllocator( .build() kubernetesClient.pods().create(podWithAttachedContainer) newlyCreatedExecutors(newExecutorId) = clock.getTimeMillis() + logDebug(s"Requested executor with id $newExecutorId from Kubernetes.") } } else if (currentRunningExecutors >= currentTotalExpectedExecutors) { // TODO handle edge cases if we end up with more running executors than expected. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala index 09477a8886421..b28d93990313e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala @@ -24,6 +24,7 @@ import scala.collection.mutable import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.ExecutorExited import org.apache.spark.util.Utils @@ -36,7 +37,7 @@ private[spark] class ExecutorPodsLifecycleManager( // job-breaking if we remove executors more than once but it's ideal if we make an attempt // to avoid doing so. Expire cache entries so that this data structure doesn't grow beyond // bounds. - removedExecutorsCache: Cache[java.lang.Long, java.lang.Long]) { + removedExecutorsCache: Cache[java.lang.Long, java.lang.Long]) extends Logging { import ExecutorPodsLifecycleManager._ @@ -55,12 +56,19 @@ private[spark] class ExecutorPodsLifecycleManager( snapshots.foreach { snapshot => snapshot.executorPods.foreach { case (execId, state) => state match { - case deleted@PodDeleted(pod) => + case deleted@PodDeleted(_) => + logDebug(s"Snapshot reported deleted executor with id $execId," + + s" pod name ${state.pod.getMetadata.getName}") removeExecutorFromSpark(schedulerBackend, deleted, execId) execIdsRemovedInThisRound += execId - case failed@PodFailed(pod) => + case failed@PodFailed(_) => + logDebug(s"Snapshot reported failed executor with id $execId," + + s" pod name ${state.pod.getMetadata.getName}") onFinalNonDeletedState(failed, execId, schedulerBackend, execIdsRemovedInThisRound) - case succeeded@PodSucceeded(pod) => + case succeeded@PodSucceeded(_) => + logDebug(s"Snapshot reported succeeded executor with id $execId," + + s" pod name ${state.pod.getMetadata.getName}. Note that succeeded executors are" + + s" unusual unless Spark specifically informed the executor to exit.") onFinalNonDeletedState(succeeded, execId, schedulerBackend, execIdsRemovedInThisRound) case _ => } @@ -78,17 +86,21 @@ private[spark] class ExecutorPodsLifecycleManager( -- latestSnapshot.executorPods.keySet -- execIdsRemovedInThisRound).foreach { missingExecutorId => if (removedExecutorsCache.getIfPresent(missingExecutorId) == null) { + val exitReasonMessage = s"The executor with ID $missingExecutorId was not found in the" + + s" cluster but we didn't get a reason why. Marking the executor as failed. The" + + s" executor may have been deleted but the driver missed the deletion event." + logDebug(exitReasonMessage) val exitReason = ExecutorExited( UNKNOWN_EXIT_CODE, exitCausedByApp = false, - s"The executor with ID $missingExecutorId was not found in the cluster but we didn't" + - s" get a reason why. Marking the executor as failed. The executor may have been" + - s" deleted but the driver missed the deletion event.") + exitReasonMessage) schedulerBackend.doRemoveExecutor(missingExecutorId.toString, exitReason) execIdsRemovedInThisRound += missingExecutorId } } } + logDebug(s"Removed executors with ids ${execIdsRemovedInThisRound.mkString(",")}" + + s" from Spark that were either found to be deleted or non-existent in the cluster.") } private def onFinalNonDeletedState( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala index 028aa01ac9324..e77e604d00e0f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala @@ -24,13 +24,14 @@ import scala.collection.JavaConverters._ import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.internal.Logging import org.apache.spark.util.ThreadUtils private[spark] class ExecutorPodsPollingSnapshotSource( conf: SparkConf, kubernetesClient: KubernetesClient, snapshotsStore: ExecutorPodsSnapshotsStore, - pollingExecutor: ScheduledExecutorService) { + pollingExecutor: ScheduledExecutorService) extends Logging { private val pollingInterval = conf.get(KUBERNETES_EXECUTOR_API_POLLING_INTERVAL) @@ -38,6 +39,7 @@ private[spark] class ExecutorPodsPollingSnapshotSource( def start(applicationId: String): Unit = { require(pollingFuture == null, "Cannot start polling more than once.") + logDebug(s"Starting to check for executor pod state every $pollingInterval ms.") pollingFuture = pollingExecutor.scheduleWithFixedDelay( new PollRunnable(applicationId), pollingInterval, pollingInterval, TimeUnit.MILLISECONDS) } @@ -52,6 +54,7 @@ private[spark] class ExecutorPodsPollingSnapshotSource( private class PollRunnable(applicationId: String) extends Runnable { override def run(): Unit = { + logDebug(s"Resynchronizing full executor pod state from Kubernetes.") snapshotsStore.replaceSnapshot(kubernetesClient .pods() .withLabel(SPARK_APP_ID_LABEL, applicationId) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala index 8776a525a084a..5583b4617eeb2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala @@ -25,6 +25,28 @@ import scala.collection.mutable import org.apache.spark.util.{ThreadUtils, Utils} +/** + * Controls the propagation of the Spark application's executor pods state to subscribers that + * react to that state. + *
+ * Roughly follows a producer-consumer model. Producers report states of executor pods, and these + * states are then published to consumers that can perform any actions in response to these states. + *
+ * Producers push updates in one of two ways. An incremental update sent by updatePod() represents + * a known new state of a single executor pod. A full sync sent by replaceSnapshot() indicates that + * the passed pods are all of the most up to date states of all executor pods for the application. + * The combination of the states of all executor pods for the application is collectively known as + * a snapshot. The store keeps track of the most up to date snapshot, and applies updates to that + * most recent snapshot - either by incrementally updating the snapshot with a single new pod state, + * or by replacing the snapshot entirely on a full sync. + *
+ * Consumers, or subscribers, register that they want to be informed about all snapshots of the + * executor pods. Every time the store replaces its most up to date snapshot from either an + * incremental update or a full sync, the most recent snapshot after the update is posted to the + * subscriber's buffer. Subscribers receive blocks of snapshots produced by the producers in + * time-windowed chunks. Each subscriber can choose to receive their snapshot chunks at different + * time intervals. + */ private[spark] class ExecutorPodsSnapshotsStoreImpl(subscribersExecutor: ScheduledExecutorService) extends ExecutorPodsSnapshotsStore { @@ -41,6 +63,9 @@ private[spark] class ExecutorPodsSnapshotsStoreImpl(subscribersExecutor: Schedul (onNewSnapshots: Seq[ExecutorPodsSnapshot] => Unit): Unit = { val newSubscriber = SnapshotsSubscriber( new LinkedBlockingQueue[ExecutorPodsSnapshot](), onNewSnapshots) + SNAPSHOT_LOCK.synchronized { + newSubscriber.snapshotsBuffer.add(currentSnapshot) + } subscribers += newSubscriber pollingTasks += subscribersExecutor.scheduleWithFixedDelay( toRunnable(() => callSubscriber(newSubscriber)), diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala index 4dfea8fad94f6..a6749a644e00c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala @@ -34,6 +34,8 @@ private[spark] class ExecutorPodsWatchSnapshotSource( def start(applicationId: String): Unit = { require(watchConnection == null, "Cannot start the watcher twice.") + logDebug(s"Starting watch for pods with labels $SPARK_APP_ID_LABEL=$applicationId," + + s" $SPARK_ROLE_LABEL=$SPARK_POD_EXECUTOR_ROLE.") watchConnection = kubernetesClient.pods() .withLabel(SPARK_APP_ID_LABEL, applicationId) .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) @@ -51,6 +53,8 @@ private[spark] class ExecutorPodsWatchSnapshotSource( private class ExecutorPodsWatcher extends Watcher[Pod] { override def eventReceived(action: Action, pod: Pod): Unit = { + val podName = pod.getMetadata.getName + logDebug(s"Received executor pod update for pod named $podName, action $action") snapshotsStore.updatePod(pod) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index cd10890fbbd1a..c6e931a38405f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -61,9 +61,10 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( "kubernetes-executor-requests") - val bufferSnapshotsExecutor = ThreadUtils - .newDaemonSingleThreadScheduledExecutor("kubernetes-executor-snapshots-buffer") - val snapshotsStore = new ExecutorPodsSnapshotsStoreImpl(bufferSnapshotsExecutor) + val subscribersExecutor = ThreadUtils + .newDaemonThreadPoolScheduledExecutor( + "kubernetes-executor-snapshots-subscribers", 2) + val snapshotsStore = new ExecutorPodsSnapshotsStoreImpl(subscribersExecutor) val removedExecutorsCache = CacheBuilder.newBuilder() .expireAfterWrite(3, TimeUnit.MINUTES) .build[java.lang.Long, java.lang.Long]() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala index 87e0b03607f70..cf54b3c4eb329 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreSuite.scala @@ -46,41 +46,47 @@ class ExecutorPodsSnapshotsStoreSuite extends SparkFunSuite with BeforeAndAfter eventQueueUnderTest.addSubscriber(2000) { receivedSnapshots2 ++= _ } + eventBufferScheduler.runUntilIdle() + assert(receivedSnapshots1 === Seq(ExecutorPodsSnapshot())) + assert(receivedSnapshots2 === Seq(ExecutorPodsSnapshot())) pushPodWithIndex(1) - assert(receivedSnapshots1.isEmpty) - assert(receivedSnapshots2.isEmpty) // Force time to move forward so that the buffer is emitted, scheduling the // processing task on the subscription executor... eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) // ... then actually execute the subscribers. assert(receivedSnapshots1 === Seq( + ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))))) - assert(receivedSnapshots2.isEmpty) + assert(receivedSnapshots2 === Seq(ExecutorPodsSnapshot())) eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) // Don't repeat snapshots assert(receivedSnapshots1 === Seq( + ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))))) assert(receivedSnapshots2 === Seq( - ExecutorPodsSnapshot(Seq(podWithIndex(1)))) - ) + ExecutorPodsSnapshot(), + ExecutorPodsSnapshot(Seq(podWithIndex(1))))) pushPodWithIndex(2) pushPodWithIndex(3) eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) assert(receivedSnapshots1 === Seq( + ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))), ExecutorPodsSnapshot(Seq(podWithIndex(1), podWithIndex(2))), ExecutorPodsSnapshot(Seq(podWithIndex(1), podWithIndex(2), podWithIndex(3))))) assert(receivedSnapshots2 === Seq( + ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))))) eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) assert(receivedSnapshots1 === Seq( + ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))), ExecutorPodsSnapshot(Seq(podWithIndex(1), podWithIndex(2))), ExecutorPodsSnapshot(Seq(podWithIndex(1), podWithIndex(2), podWithIndex(3))))) @@ -94,7 +100,7 @@ class ExecutorPodsSnapshotsStoreSuite extends SparkFunSuite with BeforeAndAfter } assert(receivedInitialSnapshot.get == null) eventBufferScheduler.runUntilIdle() - assert(receivedInitialSnapshot.get.isEmpty) + assert(receivedInitialSnapshot.get === Seq(ExecutorPodsSnapshot())) } test("Replacing the snapshot passes the new snapshot to subscribers.") { @@ -105,10 +111,12 @@ class ExecutorPodsSnapshotsStoreSuite extends SparkFunSuite with BeforeAndAfter eventQueueUnderTest.updatePod(podWithIndex(1)) eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) assert(receivedSnapshots === Seq( + ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))))) eventQueueUnderTest.replaceSnapshot(Seq(podWithIndex(2))) eventBufferScheduler.tick(1000, TimeUnit.MILLISECONDS) assert(receivedSnapshots === Seq( + ExecutorPodsSnapshot(), ExecutorPodsSnapshot(Seq(podWithIndex(1))), ExecutorPodsSnapshot(Seq(podWithIndex(2))))) }