This repository was archived by the owner on Jan 9, 2020. It is now read-only.
forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 117
Dispatch tasks to right executors that have tasks' input HDFS data #216
Merged
ash211
merged 21 commits into
apache-spark-on-k8s:branch-2.1-kubernetes
from
kimoonkim:dispatch-tasks-by-hdfs-node-locality
May 10, 2017
Merged
Changes from all commits
Commits
Show all changes
21 commits
Select commit
Hold shift + click to select a range
200ce24
Dispatch tasks to right executors that have tasks' input HDFS data on…
kimoonkim 7499e3b
Fix style issues
kimoonkim 66e79d6
Clean up unnecessary fields
kimoonkim 46f1140
Clean up a misleading method name
kimoonkim 23d287f
Address review comments
kimoonkim a026cc1
Sync and resolve conflict
kimoonkim f56f3f9
Fix import ordering
kimoonkim 177e1eb
Delete executor pods in watcher
kimoonkim a94522a
Fix the driver hang by unblocking the main thread
kimoonkim 4a7738e
Fix import order
kimoonkim 2aa7c6a
Merge branch 'branch-2.1-kubernetes' into dispatch-tasks-by-hdfs-node…
ash211 a772e7f
Merge branch 'branch-2.1-kubernetes' into dispatch-tasks-by-hdfs-node…
kimoonkim 6b1e4b4
Merge remote-tracking branch 'origin/dispatch-tasks-by-hdfs-node-loca…
kimoonkim fef7ebc
Clear runningExecutorPods
kimoonkim e07b084
Merge branch 'branch-2.1-kubernetes' into dispatch-tasks-by-hdfs-node…
ash211 b3855d6
Fix incorrect merge
ash211 7085995
Address review comments
kimoonkim ee958b3
Merge remote-tracking branch 'origin/dispatch-tasks-by-hdfs-node-loca…
kimoonkim dc0755a
Clean up imports
kimoonkim 4ce3066
Merge branch 'branch-2.1-kubernetes' into dispatch-tasks-by-hdfs-node…
ash211 80decdc
Merge branch 'branch-2.1-kubernetes' into dispatch-tasks-by-hdfs-node…
ash211 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,13 +16,18 @@ | |
| */ | ||
| package org.apache.spark.scheduler.cluster.kubernetes | ||
|
|
||
| import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} | ||
| import java.io.Closeable | ||
| import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} | ||
|
|
||
| import io.fabric8.kubernetes.api.model.{ContainerPortBuilder, EnvVarBuilder, | ||
| EnvVarSourceBuilder, Pod, QuantityBuilder} | ||
| import scala.collection.JavaConverters._ | ||
| import scala.collection.mutable | ||
| import scala.concurrent.{ExecutionContext, Future} | ||
|
|
||
| import io.fabric8.kubernetes.api.model.{ContainerPortBuilder, EnvVarBuilder, | ||
| EnvVarSourceBuilder, Pod, QuantityBuilder} | ||
| import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} | ||
| import io.fabric8.kubernetes.client.Watcher.Action | ||
|
|
||
| import org.apache.spark.{SparkContext, SparkException} | ||
| import org.apache.spark.deploy.kubernetes.config._ | ||
| import org.apache.spark.deploy.kubernetes.constants._ | ||
|
|
@@ -38,8 +43,11 @@ private[spark] class KubernetesClusterSchedulerBackend( | |
|
|
||
| import KubernetesClusterSchedulerBackend._ | ||
|
|
||
| private val EXECUTOR_MODIFICATION_LOCK = new Object | ||
| private val runningExecutorPods = new scala.collection.mutable.HashMap[String, Pod] | ||
| private val RUNNING_EXECUTOR_PODS_LOCK = new Object | ||
| private val runningExecutorPods = new mutable.HashMap[String, Pod] // Indexed by executor IDs. | ||
|
|
||
| private val EXECUTOR_PODS_BY_IPS_LOCK = new Object | ||
| private val executorPodsByIPs = new mutable.HashMap[String, Pod] // Indexed by executor IP addrs. | ||
|
|
||
| private val executorDockerImage = conf.get(EXECUTOR_DOCKER_IMAGE) | ||
| private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE) | ||
|
|
@@ -87,6 +95,7 @@ private[spark] class KubernetesClusterSchedulerBackend( | |
| super.minRegisteredRatio | ||
| } | ||
|
|
||
| private val executorWatchResource = new AtomicReference[Closeable] | ||
| protected var totalExpectedExecutors = new AtomicInteger(0) | ||
|
|
||
| private val driverUrl = RpcEndpointAddress( | ||
|
|
@@ -119,6 +128,8 @@ private[spark] class KubernetesClusterSchedulerBackend( | |
|
|
||
| override def start(): Unit = { | ||
| super.start() | ||
| executorWatchResource.set(kubernetesClient.pods().withLabel(SPARK_APP_ID_LABEL, applicationId()) | ||
| .watch(new ExecutorPodsWatcher())) | ||
| if (!Utils.isDynamicAllocationEnabled(sc.conf)) { | ||
| doRequestTotalExecutors(initialExecutors) | ||
| } | ||
|
|
@@ -133,11 +144,22 @@ private[spark] class KubernetesClusterSchedulerBackend( | |
| // When using Utils.tryLogNonFatalError some of the code fails but without any logs or | ||
| // indication as to why. | ||
| try { | ||
| runningExecutorPods.values.foreach(kubernetesClient.pods().delete(_)) | ||
| RUNNING_EXECUTOR_PODS_LOCK.synchronized { | ||
| runningExecutorPods.values.foreach(kubernetesClient.pods().delete(_)) | ||
| runningExecutorPods.clear() | ||
| } | ||
| EXECUTOR_PODS_BY_IPS_LOCK.synchronized { | ||
| executorPodsByIPs.clear() | ||
| } | ||
| val resource = executorWatchResource.getAndSet(null) | ||
| if (resource != null) { | ||
| resource.close() | ||
| } | ||
| } catch { | ||
| case e: Throwable => logError("Uncaught exception while shutting down controllers.", e) | ||
| } | ||
| try { | ||
| logInfo("Closing kubernetes client") | ||
| kubernetesClient.close() | ||
| } catch { | ||
| case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e) | ||
|
|
@@ -231,7 +253,7 @@ private[spark] class KubernetesClusterSchedulerBackend( | |
| } | ||
|
|
||
| override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { | ||
| EXECUTOR_MODIFICATION_LOCK.synchronized { | ||
| RUNNING_EXECUTOR_PODS_LOCK.synchronized { | ||
| if (requestedTotal > totalExpectedExecutors.get) { | ||
| logInfo(s"Requesting ${requestedTotal - totalExpectedExecutors.get}" | ||
| + s" additional executors, expecting total $requestedTotal and currently" + | ||
|
|
@@ -246,7 +268,7 @@ private[spark] class KubernetesClusterSchedulerBackend( | |
| } | ||
|
|
||
| override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { | ||
| EXECUTOR_MODIFICATION_LOCK.synchronized { | ||
| RUNNING_EXECUTOR_PODS_LOCK.synchronized { | ||
| for (executor <- executorIds) { | ||
| runningExecutorPods.remove(executor) match { | ||
| case Some(pod) => kubernetesClient.pods().delete(pod) | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. push the
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. |
||
|
|
@@ -256,6 +278,41 @@ private[spark] class KubernetesClusterSchedulerBackend( | |
| } | ||
| true | ||
| } | ||
|
|
||
| def getExecutorPodByIP(podIP: String): Option[Pod] = { | ||
| EXECUTOR_PODS_BY_IPS_LOCK.synchronized { | ||
| executorPodsByIPs.get(podIP) | ||
| } | ||
| } | ||
|
|
||
| private class ExecutorPodsWatcher extends Watcher[Pod] { | ||
|
|
||
| override def eventReceived(action: Action, pod: Pod): Unit = { | ||
| if (action == Action.MODIFIED && pod.getStatus.getPhase == "Running" | ||
| && pod.getMetadata.getDeletionTimestamp == null) { | ||
| val podIP = pod.getStatus.getPodIP | ||
| val clusterNodeName = pod.getSpec.getNodeName | ||
| logDebug(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.") | ||
| EXECUTOR_PODS_BY_IPS_LOCK.synchronized { | ||
| executorPodsByIPs += ((podIP, pod)) | ||
| } | ||
| } else if ((action == Action.MODIFIED && pod.getMetadata.getDeletionTimestamp != null) || | ||
| action == Action.DELETED || action == Action.ERROR) { | ||
| val podName = pod.getMetadata.getName | ||
| val podIP = pod.getStatus.getPodIP | ||
| logDebug(s"Executor pod $podName at IP $podIP was at $action.") | ||
| if (podIP != null) { | ||
| EXECUTOR_PODS_BY_IPS_LOCK.synchronized { | ||
| executorPodsByIPs -= podIP | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| override def onClose(cause: KubernetesClientException): Unit = { | ||
| logDebug("Executor pod watch closed.", cause) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private object KubernetesClusterSchedulerBackend { | ||
|
|
||
27 changes: 27 additions & 0 deletions
27
...ain/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,27 @@ | ||
| /* | ||
| * 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.kubernetes | ||
|
|
||
| import org.apache.spark.SparkContext | ||
| import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskSet, TaskSetManager} | ||
|
|
||
| private[spark] class KubernetesTaskSchedulerImpl(sc: SparkContext) extends TaskSchedulerImpl(sc) { | ||
|
|
||
| override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { | ||
| new KubernetesTaskSetManager(this, taskSet, maxTaskFailures) | ||
| } | ||
| } |
63 changes: 63 additions & 0 deletions
63
...c/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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.kubernetes | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
|
|
||
| import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskSet, TaskSetManager} | ||
|
|
||
| private[spark] class KubernetesTaskSetManager( | ||
| sched: TaskSchedulerImpl, | ||
| taskSet: TaskSet, | ||
| maxTaskFailures: Int) extends TaskSetManager(sched, taskSet, maxTaskFailures) { | ||
|
|
||
| /** | ||
| * Overrides the lookup to use not only the executor pod IP, but also the cluster node | ||
| * name and host IP address that the pod is running on. The base class may have populated | ||
| * the lookup target map with HDFS datanode locations if this task set reads HDFS data. | ||
| * Those datanode locations are based on cluster node names or host IP addresses. Using | ||
| * only executor pod IPs may not match them. | ||
| */ | ||
| override def getPendingTasksForHost(executorIP: String): ArrayBuffer[Int] = { | ||
| val pendingTasksExecutorIP = super.getPendingTasksForHost(executorIP) | ||
| if (pendingTasksExecutorIP.nonEmpty) { | ||
| pendingTasksExecutorIP | ||
| } else { | ||
| val backend = sched.backend.asInstanceOf[KubernetesClusterSchedulerBackend] | ||
| val pod = backend.getExecutorPodByIP(executorIP) | ||
| if (pod.nonEmpty) { | ||
| val clusterNodeName = pod.get.getSpec.getNodeName | ||
| val pendingTasksClusterNodeName = super.getPendingTasksForHost(clusterNodeName) | ||
| if (pendingTasksClusterNodeName.nonEmpty) { | ||
| logDebug(s"Got preferred task list $pendingTasksClusterNodeName for executor host " + | ||
| s"$executorIP using cluster node name $clusterNodeName") | ||
| pendingTasksClusterNodeName | ||
| } else { | ||
| val clusterNodeIP = pod.get.getStatus.getHostIP | ||
| val pendingTasksClusterNodeIP = super.getPendingTasksForHost(clusterNodeIP) | ||
| if (pendingTasksClusterNodeIP.nonEmpty) { | ||
| logDebug(s"Got preferred task list $pendingTasksClusterNodeIP for executor host " + | ||
| s"$executorIP using cluster node IP $clusterNodeIP") | ||
| } | ||
| pendingTasksClusterNodeIP | ||
| } | ||
| } else { | ||
| pendingTasksExecutorIP // Empty | ||
| } | ||
| } | ||
| } | ||
| } |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
do you need a
runningExecutorPods.clear()here to match theexecutorPodsByIPs.clear()below?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed in the latest diff.