From 56b7f7033c78eda90918107eaf0086e535f91ed4 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Mon, 4 Mar 2019 15:14:46 +0800 Subject: [PATCH 01/28] [SPARK-27038][CORE][YARN] Rack resolving takes a long time when initializing TaskSetManager --- .../spark/scheduler/TaskSchedulerImpl.scala | 2 + .../spark/scheduler/TaskSetManager.scala | 26 ++++- .../spark/scheduler/TaskSetManagerSuite.scala | 13 ++- .../scheduler/cluster/YarnRackResolver.scala | 94 +++++++++++++++++++ .../scheduler/cluster/YarnScheduler.scala | 30 +++++- 5 files changed, 156 insertions(+), 9 deletions(-) create mode 100644 resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnRackResolver.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 3f23bfe59523a..414d6e8117cd2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -814,6 +814,8 @@ private[spark] class TaskSchedulerImpl( // By default, rack is unknown def getRackForHost(value: String): Option[String] = None + def getRacksForHosts(values: List[String]): List[String] = Nil + private def waitBackendReady(): Unit = { if (backend.isReady) { return diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 453939aaf1901..9e81ab0f4cf61 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -184,11 +184,24 @@ private[spark] class TaskSetManager( t.epoch = epoch } + // An array to store preferred location and its task index + private val locationWithTaskIndex: ArrayBuffer[(String, Int)] = new ArrayBuffer[(String, Int)]() // Add all our tasks to the pending lists. We do this in reverse order // of task index so that tasks with low indices get launched first. for (i <- (0 until numTasks).reverse) { - addPendingTask(i) + addPendingTask(i, true) } + // SPARK-27038. Convert preferred location list to rack list in one invocation and zip with the origin index + private val rackWithTaskIndex = sched.getRacksForHosts(locationWithTaskIndex.map(_._1).toList) + .zip(locationWithTaskIndex.map(_._2)) + for ((rack, index) <- rackWithTaskIndex) { + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index + } + logInfo(s"Total pending tasks for executor are ${pendingTasksForExecutor.values.sum}, " + + s"total pending tasks for host are ${pendingTasksForHost.values.sum}, " + + s"total pending tasks for rack are ${pendingTasksForRack.values.sum}, " + + s"total pending tasks with no prefs are ${pendingTasksWithNoPrefs.length}, " + + s"total pending tasks are ${allPendingTasks.length}") /** * Track the set of locality levels which are valid given the tasks locality preferences and @@ -214,7 +227,7 @@ private[spark] class TaskSetManager( private[scheduler] var emittedTaskSizeWarning = false /** Add a task to all the pending-task lists that it should be on. */ - private[spark] def addPendingTask(index: Int) { + private[spark] def addPendingTask(index: Int, initialing: Boolean = false) { for (loc <- tasks(index).preferredLocations) { loc match { case e: ExecutorCacheTaskLocation => @@ -234,8 +247,13 @@ private[spark] class TaskSetManager( case _ => } pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer) += index - for (rack <- sched.getRackForHost(loc.host)) { - pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index + + if (initialing) { + locationWithTaskIndex += ((loc.host, index)) + } else { + for (rack <- sched.getRackForHost(loc.host)) { + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index + } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 32a2bdbae7ee4..9f7de21827405 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,7 +22,7 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.mockito.ArgumentMatchers.{any, anyInt, anyString} +import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString} import org.mockito.Mockito.{mock, never, spy, times, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -81,6 +81,10 @@ object FakeRackUtil { def getRackForHost(host: String): Option[String] = { hostToRack.get(host) } + + def getRacksForHosts(hosts: List[String]): List[String] = { + hosts.flatMap(hostToRack.get) + } } /** @@ -146,6 +150,9 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex override def getRackForHost(value: String): Option[String] = FakeRackUtil.getRackForHost(value) + + override def getRacksForHosts(values: List[String]): List[String] = + FakeRackUtil.getRacksForHosts(values) } /** @@ -1316,7 +1323,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY) // Assert the task has been black listed on the executor it was last executed on. - when(taskSetManagerSpy.addPendingTask(anyInt())).thenAnswer( + when(taskSetManagerSpy.addPendingTask(anyInt(), anyBoolean())).thenAnswer( new Answer[Unit] { override def answer(invocationOnMock: InvocationOnMock): Unit = { val task: Int = invocationOnMock.getArgument(0) @@ -1330,7 +1337,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val e = new ExceptionFailure("a", "b", Array(), "c", None) taskSetManagerSpy.handleFailedTask(taskDesc.get.taskId, TaskState.FAILED, e) - verify(taskSetManagerSpy, times(1)).addPendingTask(anyInt()) + verify(taskSetManagerSpy, times(1)).addPendingTask(anyInt(), anyBoolean()) } test("SPARK-21563 context's added jars shouldn't change mid-TaskSet") { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnRackResolver.scala new file mode 100644 index 0000000000000..f2e98b1be333f --- /dev/null +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnRackResolver.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import com.google.common.base.Strings +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.CommonConfigurationKeysPublic +import org.apache.hadoop.net._ +import org.apache.hadoop.util.ReflectionUtils + +import org.apache.spark.internal.Logging + +/** + * Added in SPARK-27038. Before using the higher Hadoop version which applied YARN-9332, + * we construct [[YarnRackResolver]] instead of [[org.apache.hadoop.yarn.util.RackResolver]] + * to revolve the rack info. + */ +object YarnRackResolver extends Logging { + private var dnsToSwitchMapping: DNSToSwitchMapping = _ + private var initCalled = false + // advisory count of arguments for rack script + private val ADVISORY_MINIMUM_NUMBER_SCRIPT_ARGS = 10000 + + def init(conf: Configuration): Unit = { + if (!initCalled) { + initCalled = true + val dnsToSwitchMappingClass = + conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + classOf[ScriptBasedMapping], classOf[DNSToSwitchMapping]) + if (classOf[ScriptBasedMapping].isAssignableFrom(dnsToSwitchMappingClass)) { + val numArgs = conf.getInt(CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY, + CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_DEFAULT) + if (numArgs < ADVISORY_MINIMUM_NUMBER_SCRIPT_ARGS) { + logWarning(s"Increasing the value of" + + s" ${CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY} could reduce" + + s" the time of rack resolving when submits a stage with a mass of tasks." + + s" Current number is $numArgs") + } + } + try { + val newInstance = ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) + .asInstanceOf[DNSToSwitchMapping] + dnsToSwitchMapping = newInstance match { + case _: CachedDNSToSwitchMapping => newInstance + case _ => new CachedDNSToSwitchMapping(newInstance) + } + } catch { + case e: Exception => + throw new RuntimeException(e) + } + } + } + + def resolveRacks(conf: Configuration, hostNames: List[String]): List[Node] = { + init(conf) + val nodes = new ArrayBuffer[Node] + val rNameList = dnsToSwitchMapping.resolve(hostNames.toList.asJava).asScala + if (rNameList == null || rNameList.isEmpty) { + hostNames.foreach(nodes += new NodeBase(_, NetworkTopology.DEFAULT_RACK)) + logInfo(s"Got an error when resolve hostNames. " + + s"Falling back to ${NetworkTopology.DEFAULT_RACK} for all") + } else { + for ((hostName, rName) <- hostNames.zip(rNameList)) { + if (Strings.isNullOrEmpty(rName)) { + // fallback to use default rack + nodes += new NodeBase(hostName, NetworkTopology.DEFAULT_RACK) + logDebug(s"Could not resolve $hostName. " + + s"Falling back to ${NetworkTopology.DEFAULT_RACK}") + } else { + nodes += new NodeBase(hostName, rName) + } + } + } + nodes.toList + } +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index 029382133ddf2..57cfa65620213 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -17,10 +17,12 @@ package org.apache.spark.scheduler.cluster +import org.apache.hadoop.net.{NetworkTopology, NodeBase} import org.apache.hadoop.yarn.util.RackResolver import org.apache.log4j.{Level, Logger} import org.apache.spark._ +import org.apache.spark.internal.config.LOCALITY_WAIT import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.Utils @@ -31,9 +33,33 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s Logger.getLogger(classOf[RackResolver]).setLevel(Level.WARN) } + // Add a on-off switch to save time for rack resolving + private val skipRackResolving = sc.conf.getTimeAsMs( + "spark.locality.wait.rack", sc.conf.get(LOCALITY_WAIT).toString) == 0 + // By default, rack is unknown override def getRackForHost(hostPort: String): Option[String] = { - val host = Utils.parseHostPort(hostPort)._1 - Option(RackResolver.resolve(sc.hadoopConfiguration, host).getNetworkLocation) + if (skipRackResolving) { + Option(NetworkTopology.DEFAULT_RACK) + } else { + val host = Utils.parseHostPort(hostPort)._1 + Option(RackResolver.resolve(sc.hadoopConfiguration, host).getNetworkLocation) + } + } + + /** + * Get racks info for a list of host. + * Use [[YarnRackResolver]] to resolve racks instead of [[RackResolver]] + * before the dependency Hadoop applied YARN-9332 + * @param hostPorts host list to resolve + * @return rack list + */ + override def getRacksForHosts(hostPorts: List[String]): List[String] = { + val hosts = hostPorts.map(Utils.parseHostPort(_)._1) + if (skipRackResolving) { + hosts.map(new NodeBase(_, NetworkTopology.DEFAULT_RACK)).map(_.getNetworkLocation) + } else { + YarnRackResolver.resolveRacks(sc.hadoopConfiguration, hosts).map(_.getNetworkLocation) + } } } From 342a6fdab3d58ca1796bfe52a22d0cb629b1d6dc Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Mon, 4 Mar 2019 20:05:07 +0800 Subject: [PATCH 02/28] log elapsed time for adding pending task --- .../org/apache/spark/scheduler/TaskSetManager.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 9e81ab0f4cf61..ca59f28c1368e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -186,22 +186,20 @@ private[spark] class TaskSetManager( // An array to store preferred location and its task index private val locationWithTaskIndex: ArrayBuffer[(String, Int)] = new ArrayBuffer[(String, Int)]() + private val addTaskStartTime = System.nanoTime() // Add all our tasks to the pending lists. We do this in reverse order // of task index so that tasks with low indices get launched first. for (i <- (0 until numTasks).reverse) { addPendingTask(i, true) } - // SPARK-27038. Convert preferred location list to rack list in one invocation and zip with the origin index + // Convert preferred location list to rack list in one invocation and zip with the origin index private val rackWithTaskIndex = sched.getRacksForHosts(locationWithTaskIndex.map(_._1).toList) .zip(locationWithTaskIndex.map(_._2)) for ((rack, index) <- rackWithTaskIndex) { pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index } - logInfo(s"Total pending tasks for executor are ${pendingTasksForExecutor.values.sum}, " + - s"total pending tasks for host are ${pendingTasksForHost.values.sum}, " + - s"total pending tasks for rack are ${pendingTasksForRack.values.sum}, " + - s"total pending tasks with no prefs are ${pendingTasksWithNoPrefs.length}, " + - s"total pending tasks are ${allPendingTasks.length}") + private val addTaskElapsedTime = (System.nanoTime() - addTaskStartTime) / 1e9 + logInfo(s"Adding pending task takes $addTaskElapsedTime seconds") /** * Track the set of locality levels which are valid given the tasks locality preferences and From 1e1180c561c693e856ee70ec3233f90f34a1212e Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Mon, 4 Mar 2019 22:00:26 +0800 Subject: [PATCH 03/28] add UT --- .../spark/scheduler/TaskSetManager.scala | 12 ++++-- .../spark/scheduler/TaskSetManagerSuite.scala | 40 +++++++++++++++++-- 2 files changed, 44 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index ca59f28c1368e..c2dad8c9d3000 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -198,7 +198,8 @@ private[spark] class TaskSetManager( for ((rack, index) <- rackWithTaskIndex) { pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index } - private val addTaskElapsedTime = (System.nanoTime() - addTaskStartTime) / 1e9 + // visible for testing + private[scheduler] val addTaskElapsedTime = (System.nanoTime() - addTaskStartTime) / 1e9 logInfo(s"Adding pending task takes $addTaskElapsedTime seconds") /** @@ -265,24 +266,27 @@ private[spark] class TaskSetManager( /** * Return the pending tasks list for a given executor ID, or an empty list if * there is no map entry for that host + * This is visible for testing. */ - private def getPendingTasksForExecutor(executorId: String): ArrayBuffer[Int] = { + private[scheduler] def getPendingTasksForExecutor(executorId: String): ArrayBuffer[Int] = { pendingTasksForExecutor.getOrElse(executorId, ArrayBuffer()) } /** * Return the pending tasks list for a given host, or an empty list if * there is no map entry for that host + * This is visible for testing. */ - private def getPendingTasksForHost(host: String): ArrayBuffer[Int] = { + private[scheduler] def getPendingTasksForHost(host: String): ArrayBuffer[Int] = { pendingTasksForHost.getOrElse(host, ArrayBuffer()) } /** * Return the pending rack-local task list for a given rack, or an empty list if * there is no map entry for that rack + * This is visible for testing. */ - private def getPendingTasksForRack(rack: String): ArrayBuffer[Int] = { + private[scheduler] def getPendingTasksForRack(rack: String): ArrayBuffer[Int] = { pendingTasksForRack.getOrElse(rack, ArrayBuffer()) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 9f7de21827405..ddedc672a10e0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -78,11 +78,17 @@ object FakeRackUtil { hostToRack(host) = rack } - def getRackForHost(host: String): Option[String] = { + def getRackForHost(host: String, slow: Boolean = false): Option[String] = { + if (slow) { + Thread.sleep(100) // assume resolving one host takes 100 ms + } hostToRack.get(host) } - def getRacksForHosts(hosts: List[String]): List[String] = { + def getRacksForHosts(hosts: List[String], slow: Boolean = false): List[String] = { + if (slow) { + Thread.sleep(500) // assume resolving multiple hosts takes 500 ms + } hosts.flatMap(hostToRack.get) } } @@ -101,6 +107,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val finishedManagers = new ArrayBuffer[TaskSetManager] val taskSetsFailed = new ArrayBuffer[String] val speculativeTasks = new ArrayBuffer[Int] + var slowRackResolve = false val executors = new mutable.HashMap[String, String] for ((execId, host) <- liveExecutors) { @@ -149,10 +156,11 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex } - override def getRackForHost(value: String): Option[String] = FakeRackUtil.getRackForHost(value) + override def getRackForHost(value: String): Option[String] = + FakeRackUtil.getRackForHost(value, slowRackResolve) override def getRacksForHosts(values: List[String]): List[String] = - FakeRackUtil.getRacksForHosts(values) + FakeRackUtil.getRacksForHosts(values, slowRackResolve) } /** @@ -1609,4 +1617,28 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg verify(sched.dagScheduler).taskEnded(manager.tasks(3), Success, result.value(), result.accumUpdates, info3) } + + test("SPARK-27038: Verify the rack resolving time and result when initialing TaskSetManager") { + sc = new SparkContext("local", "test") + for (i <- 1 to 100) { + FakeRackUtil.assignHostToRack("host" + i, "rack" + i) + } + sched = new FakeTaskScheduler(sc, + ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + sched.slowRackResolve = true + val locations = new ArrayBuffer[Seq[TaskLocation]]() + for (i <- 1 to 100) { + locations += Seq(TaskLocation("host" + i)) + } + val taskSet = FakeTask.createTaskSet(100, locations: _*) + val clock = new ManualClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + var total = 0 + for (i <- 1 to 100) { + total += manager.getPendingTasksForRack("rack" + i).length + } + assert(total === 100) // verify the total number always equals 100 with/without SPARK-27038 + // verify elapsed time should be less than 1s, without SPARK-27038, it should be larger 10s + assert(manager.addTaskElapsedTime < 1) + } } From e521bcfd285e10f58a860c7ec3634fb286a74cdf Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Thu, 7 Mar 2019 19:20:14 +0800 Subject: [PATCH 04/28] address comments --- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 14 +-- .../spark/scheduler/TaskSetManagerSuite.scala | 49 +++++++--- .../spark/deploy/yarn/SparkRackResolver.scala | 77 +++++++++++++++ .../scheduler/cluster/YarnRackResolver.scala | 94 ------------------- .../scheduler/cluster/YarnScheduler.scala | 13 ++- 6 files changed, 127 insertions(+), 122 deletions(-) delete mode 100644 resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnRackResolver.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 414d6e8117cd2..c5ab3816d29ee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -814,7 +814,7 @@ private[spark] class TaskSchedulerImpl( // By default, rack is unknown def getRackForHost(value: String): Option[String] = None - def getRacksForHosts(values: List[String]): List[String] = Nil + def getRacksForHosts(values: List[String]): List[Option[String]] = values.map(getRackForHost) private def waitBackendReady(): Unit = { if (backend.isReady) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index c2dad8c9d3000..3c388aab4525e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -195,12 +195,12 @@ private[spark] class TaskSetManager( // Convert preferred location list to rack list in one invocation and zip with the origin index private val rackWithTaskIndex = sched.getRacksForHosts(locationWithTaskIndex.map(_._1).toList) .zip(locationWithTaskIndex.map(_._2)) - for ((rack, index) <- rackWithTaskIndex) { - pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index + rackWithTaskIndex.foreach { + case (Some(rack), index) => + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index + case _ => } - // visible for testing - private[scheduler] val addTaskElapsedTime = (System.nanoTime() - addTaskStartTime) / 1e9 - logInfo(s"Adding pending task takes $addTaskElapsedTime seconds") + logInfo(s"Adding pending tasks take ${(System.nanoTime() - addTaskStartTime) / 1e9} seconds") /** * Track the set of locality levels which are valid given the tasks locality preferences and @@ -226,7 +226,7 @@ private[spark] class TaskSetManager( private[scheduler] var emittedTaskSizeWarning = false /** Add a task to all the pending-task lists that it should be on. */ - private[spark] def addPendingTask(index: Int, initialing: Boolean = false) { + private[spark] def addPendingTask(index: Int, initializing: Boolean = false) { for (loc <- tasks(index).preferredLocations) { loc match { case e: ExecutorCacheTaskLocation => @@ -247,7 +247,7 @@ private[spark] class TaskSetManager( } pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer) += index - if (initialing) { + if (initializing) { locationWithTaskIndex += ((loc.host, index)) } else { for (rack <- sched.getRackForHost(loc.host)) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index ddedc672a10e0..7686fd2a8273d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -69,27 +69,47 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) // Get the rack for a given host object FakeRackUtil { private val hostToRack = new mutable.HashMap[String, String]() + var loopCount = 0 def cleanUp() { hostToRack.clear() + loopCount = 0 } def assignHostToRack(host: String, rack: String) { hostToRack(host) = rack } - def getRackForHost(host: String, slow: Boolean = false): Option[String] = { - if (slow) { - Thread.sleep(100) // assume resolving one host takes 100 ms - } + def getRackForHost(host: String): Option[String] = { + loopCount = simulateRunResolveCommand(Seq(host)) hostToRack.get(host) } - def getRacksForHosts(hosts: List[String], slow: Boolean = false): List[String] = { - if (slow) { - Thread.sleep(500) // assume resolving multiple hosts takes 500 ms + def getRacksForHosts(hosts: List[String]): List[Option[String]] = { + loopCount = simulateRunResolveCommand(hosts) + hosts.map(hostToRack.get) + } + + /** + * This is a simulation of building and executing the resolution command. + * Simulate function `runResolveCommand()` in [[org.apache.hadoop.net.ScriptBasedMapping]]. + * If Seq has 100 elements, it returns 4. If Seq has 1 elements, it returns 1. + * @param args a list of arguments + * @return script execution times + */ + private def simulateRunResolveCommand(args: Seq[String]): Int = { + val maxArgs = 30 // Simulate NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_DEFAULT + var numProcessed = 0 + var loopCount = 0 + while (numProcessed != args.size) { + var start = maxArgs * loopCount + numProcessed = start + while (numProcessed < (start + maxArgs) && numProcessed < args.size) { + numProcessed += 1 + } + loopCount += 1 } - hosts.flatMap(hostToRack.get) + loopCount } } @@ -157,10 +177,10 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex override def getRackForHost(value: String): Option[String] = - FakeRackUtil.getRackForHost(value, slowRackResolve) + FakeRackUtil.getRackForHost(value) - override def getRacksForHosts(values: List[String]): List[String] = - FakeRackUtil.getRacksForHosts(values, slowRackResolve) + override def getRacksForHosts(values: List[String]): List[Option[String]] = + FakeRackUtil.getRacksForHosts(values) } /** @@ -1618,7 +1638,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg result.accumUpdates, info3) } - test("SPARK-27038: Verify the rack resolving time and result when initialing TaskSetManager") { + test("SPARK-27038: Verify the rack resolving time has been reduced") { sc = new SparkContext("local", "test") for (i <- 1 to 100) { FakeRackUtil.assignHostToRack("host" + i, "rack" + i) @@ -1637,8 +1657,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg for (i <- 1 to 100) { total += manager.getPendingTasksForRack("rack" + i).length } - assert(total === 100) // verify the total number always equals 100 with/without SPARK-27038 - // verify elapsed time should be less than 1s, without SPARK-27038, it should be larger 10s - assert(manager.addTaskElapsedTime < 1) + assert(total === 100) // verify the total number not changed with SPARK-27038 + assert(FakeRackUtil.loopCount == 4) // verify script execution loop count decreased } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index c711d088f2116..5bcfaf8d2ca59 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -17,10 +17,18 @@ package org.apache.spark.deploy.yarn +import scala.collection.mutable.ArrayBuffer + +import com.google.common.base.Strings import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.CommonConfigurationKeysPublic +import org.apache.hadoop.net._ +import org.apache.hadoop.util.ReflectionUtils import org.apache.hadoop.yarn.util.RackResolver import org.apache.log4j.{Level, Logger} +import org.apache.spark.internal.Logging + /** * Wrapper around YARN's [[RackResolver]]. This allows Spark tests to easily override the * default behavior, since YARN's class self-initializes the first time it's called, and @@ -38,3 +46,72 @@ private[yarn] class SparkRackResolver { } } + +/** + * Utility to resolve the rack for hosts in an efficient manner. + * It will cache the rack for individual hosts to avoid + * repeatedly performing the same expensive lookup. + * + * This will be unnecessary in hadoop releases with YARN-9332. + * With that, we could just directly use [[org.apache.hadoop.yarn.util.RackResolver]]. + * In the meantime, this is a re-implementation for spark's use. + */ +object SparkRackResolver extends Logging { + private var dnsToSwitchMapping: DNSToSwitchMapping = _ + private var initCalled = false + // advisory count of arguments for rack script, less than this will get a warning + private val ADVISORY_MINIMUM_NUMBER_SCRIPT_ARGS = 10000 + + def init(conf: Configuration): Unit = { + if (!initCalled) { + initCalled = true + val dnsToSwitchMappingClass = + conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + classOf[ScriptBasedMapping], classOf[DNSToSwitchMapping]) + if (classOf[ScriptBasedMapping].isAssignableFrom(dnsToSwitchMappingClass)) { + val numArgs = conf.getInt(CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY, + CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_DEFAULT) + if (numArgs < ADVISORY_MINIMUM_NUMBER_SCRIPT_ARGS) { + logWarning(s"Increasing the value of" + + s" ${CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY} could reduce" + + s" the time of rack resolving when submits a stage with a mass of tasks." + + s" Current number is $numArgs") + } + } + try { + val newInstance = ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) + .asInstanceOf[DNSToSwitchMapping] + dnsToSwitchMapping = newInstance match { + case _: CachedDNSToSwitchMapping => newInstance + case _ => new CachedDNSToSwitchMapping(newInstance) + } + } catch { + case e: Exception => + throw new RuntimeException(e) + } + } + } + + def resolveRacks(conf: Configuration, hostNames: List[String]): List[Node] = { + init(conf) + val nodes = new ArrayBuffer[Node] + val rNameList = dnsToSwitchMapping.resolve(hostNames.toList.asJava).asScala + if (rNameList == null || rNameList.isEmpty) { + hostNames.foreach(nodes += new NodeBase(_, NetworkTopology.DEFAULT_RACK)) + logInfo(s"Got an error when resolve hostNames. " + + s"Falling back to ${NetworkTopology.DEFAULT_RACK} for all") + } else { + for ((hostName, rName) <- hostNames.zip(rNameList)) { + if (Strings.isNullOrEmpty(rName)) { + // fallback to use default rack + nodes += new NodeBase(hostName, NetworkTopology.DEFAULT_RACK) + logDebug(s"Could not resolve $hostName. " + + s"Falling back to ${NetworkTopology.DEFAULT_RACK}") + } else { + nodes += new NodeBase(hostName, rName) + } + } + } + nodes.toList + } +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnRackResolver.scala deleted file mode 100644 index f2e98b1be333f..0000000000000 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnRackResolver.scala +++ /dev/null @@ -1,94 +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 - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer - -import com.google.common.base.Strings -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.CommonConfigurationKeysPublic -import org.apache.hadoop.net._ -import org.apache.hadoop.util.ReflectionUtils - -import org.apache.spark.internal.Logging - -/** - * Added in SPARK-27038. Before using the higher Hadoop version which applied YARN-9332, - * we construct [[YarnRackResolver]] instead of [[org.apache.hadoop.yarn.util.RackResolver]] - * to revolve the rack info. - */ -object YarnRackResolver extends Logging { - private var dnsToSwitchMapping: DNSToSwitchMapping = _ - private var initCalled = false - // advisory count of arguments for rack script - private val ADVISORY_MINIMUM_NUMBER_SCRIPT_ARGS = 10000 - - def init(conf: Configuration): Unit = { - if (!initCalled) { - initCalled = true - val dnsToSwitchMappingClass = - conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, - classOf[ScriptBasedMapping], classOf[DNSToSwitchMapping]) - if (classOf[ScriptBasedMapping].isAssignableFrom(dnsToSwitchMappingClass)) { - val numArgs = conf.getInt(CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY, - CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_DEFAULT) - if (numArgs < ADVISORY_MINIMUM_NUMBER_SCRIPT_ARGS) { - logWarning(s"Increasing the value of" + - s" ${CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY} could reduce" + - s" the time of rack resolving when submits a stage with a mass of tasks." + - s" Current number is $numArgs") - } - } - try { - val newInstance = ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) - .asInstanceOf[DNSToSwitchMapping] - dnsToSwitchMapping = newInstance match { - case _: CachedDNSToSwitchMapping => newInstance - case _ => new CachedDNSToSwitchMapping(newInstance) - } - } catch { - case e: Exception => - throw new RuntimeException(e) - } - } - } - - def resolveRacks(conf: Configuration, hostNames: List[String]): List[Node] = { - init(conf) - val nodes = new ArrayBuffer[Node] - val rNameList = dnsToSwitchMapping.resolve(hostNames.toList.asJava).asScala - if (rNameList == null || rNameList.isEmpty) { - hostNames.foreach(nodes += new NodeBase(_, NetworkTopology.DEFAULT_RACK)) - logInfo(s"Got an error when resolve hostNames. " + - s"Falling back to ${NetworkTopology.DEFAULT_RACK} for all") - } else { - for ((hostName, rName) <- hostNames.zip(rNameList)) { - if (Strings.isNullOrEmpty(rName)) { - // fallback to use default rack - nodes += new NodeBase(hostName, NetworkTopology.DEFAULT_RACK) - logDebug(s"Could not resolve $hostName. " + - s"Falling back to ${NetworkTopology.DEFAULT_RACK}") - } else { - nodes += new NodeBase(hostName, rName) - } - } - } - nodes.toList - } -} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index 57cfa65620213..4239677310647 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -17,11 +17,12 @@ package org.apache.spark.scheduler.cluster -import org.apache.hadoop.net.{NetworkTopology, NodeBase} +import org.apache.hadoop.net.NetworkTopology import org.apache.hadoop.yarn.util.RackResolver import org.apache.log4j.{Level, Logger} import org.apache.spark._ +import org.apache.spark.deploy.yarn.SparkRackResolver import org.apache.spark.internal.config.LOCALITY_WAIT import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.Utils @@ -49,17 +50,19 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s /** * Get racks info for a list of host. - * Use [[YarnRackResolver]] to resolve racks instead of [[RackResolver]] + * Use [[SparkRackResolver]] to resolve racks instead of [[RackResolver]] * before the dependency Hadoop applied YARN-9332 * @param hostPorts host list to resolve * @return rack list */ - override def getRacksForHosts(hostPorts: List[String]): List[String] = { + override def getRacksForHosts(hostPorts: List[String]): List[Option[String]] = { val hosts = hostPorts.map(Utils.parseHostPort(_)._1) if (skipRackResolving) { - hosts.map(new NodeBase(_, NetworkTopology.DEFAULT_RACK)).map(_.getNetworkLocation) + hosts.map(_ => Option(NetworkTopology.DEFAULT_RACK)) } else { - YarnRackResolver.resolveRacks(sc.hadoopConfiguration, hosts).map(_.getNetworkLocation) + SparkRackResolver.resolveRacks(sc.hadoopConfiguration, hosts).map { node => + Option(node.getNetworkLocation) + } } } } From 8514b3e17a8f9fbde7a67dfe674831996722fe4a Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Fri, 8 Mar 2019 21:30:59 +0800 Subject: [PATCH 05/28] address comments --- .../spark/scheduler/TaskSetManager.scala | 45 +++++++------ .../spark/scheduler/TaskSetManagerSuite.scala | 64 +++++++++++++++---- .../spark/deploy/yarn/SparkRackResolver.scala | 27 ++++---- .../scheduler/cluster/YarnScheduler.scala | 12 ++-- 4 files changed, 95 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 3c388aab4525e..716dfe5e20eb7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -184,23 +184,25 @@ private[spark] class TaskSetManager( t.epoch = epoch } - // An array to store preferred location and its task index - private val locationWithTaskIndex: ArrayBuffer[(String, Int)] = new ArrayBuffer[(String, Int)]() - private val addTaskStartTime = System.nanoTime() // Add all our tasks to the pending lists. We do this in reverse order // of task index so that tasks with low indices get launched first. - for (i <- (0 until numTasks).reverse) { - addPendingTask(i, true) - } - // Convert preferred location list to rack list in one invocation and zip with the origin index - private val rackWithTaskIndex = sched.getRacksForHosts(locationWithTaskIndex.map(_._1).toList) - .zip(locationWithTaskIndex.map(_._2)) - rackWithTaskIndex.foreach { - case (Some(rack), index) => - pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index - case _ => + addPendingTasks() + + private def addPendingTasks(): Unit = { + val array = new ArrayBuffer[(String, Int)]() // Array(preferredLocation, task index) + val duration = Utils.timeTakenMs( + for (i <- (0 until numTasks).reverse) { + addPendingTask(i, Some(array)) + } + ) + // Convert preferred location list to rack list in one invocation and zip with the origin index + sched.getRacksForHosts(array.map(_._1).toList).zip(array.map(_._2)) foreach { + case (Some(rack), index) => + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index + case _ => + } + logInfo(s"Adding pending tasks take $duration ms") } - logInfo(s"Adding pending tasks take ${(System.nanoTime() - addTaskStartTime) / 1e9} seconds") /** * Track the set of locality levels which are valid given the tasks locality preferences and @@ -226,7 +228,8 @@ private[spark] class TaskSetManager( private[scheduler] var emittedTaskSizeWarning = false /** Add a task to all the pending-task lists that it should be on. */ - private[spark] def addPendingTask(index: Int, initializing: Boolean = false) { + private[spark] def addPendingTask(index: Int, + initializingTaskArray: Option[ArrayBuffer[(String, Int)]] = None) { for (loc <- tasks(index).preferredLocations) { loc match { case e: ExecutorCacheTaskLocation => @@ -247,12 +250,12 @@ private[spark] class TaskSetManager( } pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer) += index - if (initializing) { - locationWithTaskIndex += ((loc.host, index)) - } else { - for (rack <- sched.getRackForHost(loc.host)) { - pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index - } + initializingTaskArray match { + case Some(array) => array += ((loc.host, index)) + case None => + for (rack <- sched.getRackForHost(loc.host)) { + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index + } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 7686fd2a8273d..082e081c5cb99 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,8 +22,9 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.hadoop.net.NetworkTopology import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString} -import org.mockito.Mockito.{mock, never, spy, times, verify, when} +import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -130,6 +131,10 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex var slowRackResolve = false val executors = new mutable.HashMap[String, String] + + val skipRackResolving = sc.conf.getTimeAsMs( + "spark.locality.wait.rack", sc.conf.get(config.LOCALITY_WAIT).toString) == 0 + for ((execId, host) <- liveExecutors) { addExecutor(execId, host) } @@ -175,12 +180,20 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex } } - override def getRackForHost(value: String): Option[String] = - FakeRackUtil.getRackForHost(value) + if (skipRackResolving) { + Option(NetworkTopology.DEFAULT_RACK) + } else { + FakeRackUtil.getRackForHost(value) + } + override def getRacksForHosts(values: List[String]): List[Option[String]] = - FakeRackUtil.getRacksForHosts(values) + if (skipRackResolving) { + values.map(_ => Option(NetworkTopology.DEFAULT_RACK)) + } else { + FakeRackUtil.getRacksForHosts(values) + } } /** @@ -1351,7 +1364,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY) // Assert the task has been black listed on the executor it was last executed on. - when(taskSetManagerSpy.addPendingTask(anyInt(), anyBoolean())).thenAnswer( + when(taskSetManagerSpy.addPendingTask(anyInt(), any())).thenAnswer( new Answer[Unit] { override def answer(invocationOnMock: InvocationOnMock): Unit = { val task: Int = invocationOnMock.getArgument(0) @@ -1365,7 +1378,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val e = new ExceptionFailure("a", "b", Array(), "c", None) taskSetManagerSpy.handleFailedTask(taskDesc.get.taskId, TaskState.FAILED, e) - verify(taskSetManagerSpy, times(1)).addPendingTask(anyInt(), anyBoolean()) + verify(taskSetManagerSpy, times(1)).addPendingTask(anyInt(), any()) } test("SPARK-21563 context's added jars shouldn't change mid-TaskSet") { @@ -1638,26 +1651,51 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg result.accumUpdates, info3) } - test("SPARK-27038: Verify the rack resolving time has been reduced") { + test("SPARK-27038 Verify the rack resolving time has been reduced") { sc = new SparkContext("local", "test") - for (i <- 1 to 100) { - FakeRackUtil.assignHostToRack("host" + i, "rack" + i) + for (i <- 0 to 99) { + FakeRackUtil.assignHostToRack("host" + i, "rack" + (i % 20)) } sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) sched.slowRackResolve = true val locations = new ArrayBuffer[Seq[TaskLocation]]() - for (i <- 1 to 100) { + for (i <- 0 to 99) { locations += Seq(TaskLocation("host" + i)) } val taskSet = FakeTask.createTaskSet(100, locations: _*) val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) var total = 0 - for (i <- 1 to 100) { - total += manager.getPendingTasksForRack("rack" + i).length + for (i <- 0 until 20) { + val numTaskInRack = manager.getPendingTasksForRack("rack" + i).length + assert(numTaskInRack === 5) + total += numTaskInRack } + assert(sched.skipRackResolving === false) assert(total === 100) // verify the total number not changed with SPARK-27038 - assert(FakeRackUtil.loopCount == 4) // verify script execution loop count decreased + assert(FakeRackUtil.loopCount === 4) // verify script execution loop count decreased + } + + test("SPARK-27038 Verify the rack resolving time when spark.locality.wait is zero") { + val conf = new SparkConf().set(config.LOCALITY_WAIT.key, "0") + sc = new SparkContext("local", "test", conf) + for (i <- 0 to 99) { + FakeRackUtil.assignHostToRack("host" + i, "rack" + (i % 20)) + } + sched = new FakeTaskScheduler(sc, + ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + sched.slowRackResolve = true + val locations = new ArrayBuffer[Seq[TaskLocation]]() + for (i <- 0 to 99) { + locations += Seq(TaskLocation("host" + i)) + } + val taskSet = FakeTask.createTaskSet(100, locations: _*) + val clock = new ManualClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + assert(sched.skipRackResolving === true) + // verify the total number not changed with SPARK-27038 + assert(manager.getPendingTasksForRack(NetworkTopology.DEFAULT_RACK).length === 100) + assert(FakeRackUtil.loopCount === 0) // verify no invocation of FakeRackUtil.getRacksForHosts() } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index 5bcfaf8d2ca59..2d357f517712d 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.yarn +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import com.google.common.base.Strings @@ -34,7 +35,7 @@ import org.apache.spark.internal.Logging * default behavior, since YARN's class self-initializes the first time it's called, and * future calls all use the initial configuration. */ -private[yarn] class SparkRackResolver { +private[spark] class SparkRackResolver { // RackResolver logs an INFO message whenever it resolves a rack, which is way too often. if (Logger.getLogger(classOf[RackResolver]).getLevel == null) { @@ -45,6 +46,14 @@ private[yarn] class SparkRackResolver { RackResolver.resolve(conf, hostName).getNetworkLocation() } + /** + * Added in SPARK-27038. + * This should be changed to `RackResolver.resolve(conf, hostNames)` + * in hadoop releases with YARN-9332. + */ + def resolve(conf: Configuration, hostNames: List[String]): List[Node] = { + SparkRackResolver.resolve(conf, hostNames) + } } /** @@ -52,6 +61,7 @@ private[yarn] class SparkRackResolver { * It will cache the rack for individual hosts to avoid * repeatedly performing the same expensive lookup. * + * Its logic refers [[org.apache.hadoop.yarn.util.RackResolver]] and enhanced. * This will be unnecessary in hadoop releases with YARN-9332. * With that, we could just directly use [[org.apache.hadoop.yarn.util.RackResolver]]. * In the meantime, this is a re-implementation for spark's use. @@ -59,10 +69,8 @@ private[yarn] class SparkRackResolver { object SparkRackResolver extends Logging { private var dnsToSwitchMapping: DNSToSwitchMapping = _ private var initCalled = false - // advisory count of arguments for rack script, less than this will get a warning - private val ADVISORY_MINIMUM_NUMBER_SCRIPT_ARGS = 10000 - def init(conf: Configuration): Unit = { + private def init(conf: Configuration): Unit = { if (!initCalled) { initCalled = true val dnsToSwitchMappingClass = @@ -71,12 +79,9 @@ object SparkRackResolver extends Logging { if (classOf[ScriptBasedMapping].isAssignableFrom(dnsToSwitchMappingClass)) { val numArgs = conf.getInt(CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY, CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_DEFAULT) - if (numArgs < ADVISORY_MINIMUM_NUMBER_SCRIPT_ARGS) { - logWarning(s"Increasing the value of" + - s" ${CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY} could reduce" + - s" the time of rack resolving when submits a stage with a mass of tasks." + - s" Current number is $numArgs") - } + logInfo(s"Setting spark.hadoop.net.topology.script.number.args with a higher value " + + s"may reduce the time of rack resolving when submits a stage with a mass of tasks. " + + s"Current number is $numArgs") } try { val newInstance = ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) @@ -92,7 +97,7 @@ object SparkRackResolver extends Logging { } } - def resolveRacks(conf: Configuration, hostNames: List[String]): List[Node] = { + def resolve(conf: Configuration, hostNames: List[String]): List[Node] = { init(conf) val nodes = new ArrayBuffer[Node] val rNameList = dnsToSwitchMapping.resolve(hostNames.toList.asJava).asScala diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index 4239677310647..4c188b504a5bf 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.net.NetworkTopology import org.apache.hadoop.yarn.util.RackResolver -import org.apache.log4j.{Level, Logger} import org.apache.spark._ import org.apache.spark.deploy.yarn.SparkRackResolver @@ -29,13 +28,10 @@ import org.apache.spark.util.Utils private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(sc) { - // RackResolver logs an INFO message whenever it resolves a rack, which is way too often. - if (Logger.getLogger(classOf[RackResolver]).getLevel == null) { - Logger.getLogger(classOf[RackResolver]).setLevel(Level.WARN) - } + private[spark] val resolver = new SparkRackResolver // Add a on-off switch to save time for rack resolving - private val skipRackResolving = sc.conf.getTimeAsMs( + val skipRackResolving: Boolean = sc.conf.getTimeAsMs( "spark.locality.wait.rack", sc.conf.get(LOCALITY_WAIT).toString) == 0 // By default, rack is unknown @@ -44,7 +40,7 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s Option(NetworkTopology.DEFAULT_RACK) } else { val host = Utils.parseHostPort(hostPort)._1 - Option(RackResolver.resolve(sc.hadoopConfiguration, host).getNetworkLocation) + Option(resolver.resolve(sc.hadoopConfiguration, host)) } } @@ -60,7 +56,7 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s if (skipRackResolving) { hosts.map(_ => Option(NetworkTopology.DEFAULT_RACK)) } else { - SparkRackResolver.resolveRacks(sc.hadoopConfiguration, hosts).map { node => + resolver.resolve(sc.hadoopConfiguration, hosts).map { node => Option(node.getNetworkLocation) } } From 63666ad9cdbff408e34e7bbee5c09685f6fd1857 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Fri, 8 Mar 2019 21:44:49 +0800 Subject: [PATCH 06/28] trival fix --- .../org/apache/spark/scheduler/TaskSetManagerSuite.scala | 5 +---- .../org/apache/spark/deploy/yarn/SparkRackResolver.scala | 4 ++-- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 082e081c5cb99..0cc074ffaceb7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -128,7 +128,6 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val finishedManagers = new ArrayBuffer[TaskSetManager] val taskSetsFailed = new ArrayBuffer[String] val speculativeTasks = new ArrayBuffer[Int] - var slowRackResolve = false val executors = new mutable.HashMap[String, String] @@ -1658,7 +1657,6 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) - sched.slowRackResolve = true val locations = new ArrayBuffer[Seq[TaskLocation]]() for (i <- 0 to 99) { locations += Seq(TaskLocation("host" + i)) @@ -1669,7 +1667,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg var total = 0 for (i <- 0 until 20) { val numTaskInRack = manager.getPendingTasksForRack("rack" + i).length - assert(numTaskInRack === 5) + assert(numTaskInRack === 5) // check rack assignment is still done correctly total += numTaskInRack } assert(sched.skipRackResolving === false) @@ -1685,7 +1683,6 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) - sched.slowRackResolve = true val locations = new ArrayBuffer[Seq[TaskLocation]]() for (i <- 0 to 99) { locations += Seq(TaskLocation("host" + i)) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index 2d357f517712d..3deebb0cfe24b 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -52,7 +52,7 @@ private[spark] class SparkRackResolver { * in hadoop releases with YARN-9332. */ def resolve(conf: Configuration, hostNames: List[String]): List[Node] = { - SparkRackResolver.resolve(conf, hostNames) + SparkRackResolver.coreResolve(conf, hostNames) } } @@ -97,7 +97,7 @@ object SparkRackResolver extends Logging { } } - def resolve(conf: Configuration, hostNames: List[String]): List[Node] = { + def coreResolve(conf: Configuration, hostNames: List[String]): List[Node] = { init(conf) val nodes = new ArrayBuffer[Node] val rNameList = dnsToSwitchMapping.resolve(hostNames.toList.asJava).asScala From 69b62e4bbc96b6f9f406e60bf2ff2dd15b95af2c Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Mon, 11 Mar 2019 09:33:12 +0800 Subject: [PATCH 07/28] address comments --- .../apache/spark/internal/config/package.scala | 2 +- .../apache/spark/scheduler/TaskSetManager.scala | 16 ++++++++-------- .../spark/scheduler/TaskSetManagerSuite.scala | 3 +-- .../spark/deploy/yarn/SparkRackResolver.scala | 16 +++++----------- .../spark/scheduler/cluster/YarnScheduler.scala | 5 ++--- 5 files changed, 17 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 3ef6cba8193f6..ac9ef44d67554 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1129,7 +1129,7 @@ package object config { private[spark] val LOCALITY_WAIT_RACK = ConfigBuilder("spark.locality.wait.rack") .fallbackConf(LOCALITY_WAIT) - private[spark] val REDUCER_MAX_SIZE_IN_FLIGHT = ConfigBuilder("spark.reducer.maxSizeInFlight") + private[spark] val REDUCER_MAX_SIZE_IN_FLIGHT = ConfigBuilder("spark.reducer.maxSizeInFlight") .doc("Maximum size of map outputs to fetch simultaneously from each reduce task, " + "in MiB unless otherwise specified. Since each output requires us to create a " + "buffer to receive it, this represents a fixed memory overhead per reduce task, " + diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 716dfe5e20eb7..31fbd86f2703b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -189,17 +189,17 @@ private[spark] class TaskSetManager( addPendingTasks() private def addPendingTasks(): Unit = { - val array = new ArrayBuffer[(String, Int)]() // Array(preferredLocation, task index) - val duration = Utils.timeTakenMs( + val (_, duration) = Utils.timeTakenMs { + val array = new ArrayBuffer[(String, Int)]() // Array(preferredLocation, task index) for (i <- (0 until numTasks).reverse) { addPendingTask(i, Some(array)) } - ) - // Convert preferred location list to rack list in one invocation and zip with the origin index - sched.getRacksForHosts(array.map(_._1).toList).zip(array.map(_._2)) foreach { - case (Some(rack), index) => - pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index - case _ => + // Convert preferred locations to racks in one invocation and zip with the origin indices + sched.getRacksForHosts(array.map(_._1).toList).zip(array.map(_._2)) foreach { + case (Some(rack), index) => + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index + case _ => + } } logInfo(s"Adding pending tasks take $duration ms") } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 0cc074ffaceb7..45fd1793f5d10 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -131,8 +131,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val executors = new mutable.HashMap[String, String] - val skipRackResolving = sc.conf.getTimeAsMs( - "spark.locality.wait.rack", sc.conf.get(config.LOCALITY_WAIT).toString) == 0 + val skipRackResolving = sc.conf.get(config.LOCALITY_WAIT_RACK) == 0L for ((execId, host) <- liveExecutors) { addExecutor(execId, host) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index 3deebb0cfe24b..f16e35ef25753 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -83,16 +83,11 @@ object SparkRackResolver extends Logging { s"may reduce the time of rack resolving when submits a stage with a mass of tasks. " + s"Current number is $numArgs") } - try { - val newInstance = ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) - .asInstanceOf[DNSToSwitchMapping] - dnsToSwitchMapping = newInstance match { - case _: CachedDNSToSwitchMapping => newInstance - case _ => new CachedDNSToSwitchMapping(newInstance) - } - } catch { - case e: Exception => - throw new RuntimeException(e) + val newInstance = ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) + .asInstanceOf[DNSToSwitchMapping] + dnsToSwitchMapping = newInstance match { + case _: CachedDNSToSwitchMapping => newInstance + case _ => new CachedDNSToSwitchMapping(newInstance) } } } @@ -108,7 +103,6 @@ object SparkRackResolver extends Logging { } else { for ((hostName, rName) <- hostNames.zip(rNameList)) { if (Strings.isNullOrEmpty(rName)) { - // fallback to use default rack nodes += new NodeBase(hostName, NetworkTopology.DEFAULT_RACK) logDebug(s"Could not resolve $hostName. " + s"Falling back to ${NetworkTopology.DEFAULT_RACK}") diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index 4c188b504a5bf..09e72d3771869 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.yarn.util.RackResolver import org.apache.spark._ import org.apache.spark.deploy.yarn.SparkRackResolver -import org.apache.spark.internal.config.LOCALITY_WAIT +import org.apache.spark.internal.config.LOCALITY_WAIT_RACK import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.Utils @@ -31,8 +31,7 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s private[spark] val resolver = new SparkRackResolver // Add a on-off switch to save time for rack resolving - val skipRackResolving: Boolean = sc.conf.getTimeAsMs( - "spark.locality.wait.rack", sc.conf.get(LOCALITY_WAIT).toString) == 0 + val skipRackResolving: Boolean = sc.conf.get(LOCALITY_WAIT_RACK) == 0L // By default, rack is unknown override def getRackForHost(hostPort: String): Option[String] = { From 6246e57ae6e8cb1d37e87afd8a091d73d57d2216 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Mon, 18 Mar 2019 13:54:54 +0800 Subject: [PATCH 08/28] refactor --- .../spark/scheduler/TaskSchedulerImpl.scala | 32 +++++++++- .../spark/scheduler/TaskSetManagerSuite.scala | 63 ++++--------------- .../spark/deploy/yarn/SparkRackResolver.scala | 16 +---- .../scheduler/cluster/YarnScheduler.scala | 30 ++------- 4 files changed, 51 insertions(+), 90 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index c5ab3816d29ee..c8743f9233c20 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -811,10 +811,36 @@ private[spark] class TaskSchedulerImpl( blacklistTrackerOpt.map(_.nodeBlacklist()).getOrElse(scala.collection.immutable.Set()) } - // By default, rack is unknown - def getRackForHost(value: String): Option[String] = None + // Add a on-off switch to save time for rack resolving + def skipRackResolving: Boolean = sc.conf.get(LOCALITY_WAIT_RACK) == 0L - def getRacksForHosts(values: List[String]): List[Option[String]] = values.map(getRackForHost) + /** + * Rack is unknown by default. + * It can be override in different TaskScheduler, like Yarn. + */ + def defaultRackValue: Option[String] = None + + def doGetRacksForHosts(preferredLocation: List[String]): List[Option[String]] = Nil + + def getRackForHost(value: String): Option[String] = { + if (skipRackResolving) { + defaultRackValue + } else { + doGetRacksForHosts(List(value)) match { + case Nil => None + case h :: Nil => h + case _ => None + } + } + } + + def getRacksForHosts(values: List[String]): List[Option[String]] = { + if (skipRackResolving) { + values.map(_ => defaultRackValue) + } else { + doGetRacksForHosts(values) + } + } private def waitBackendReady(): Unit = { if (backend.isReady) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 45fd1793f5d10..759757bf7464f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,8 +22,7 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.hadoop.net.NetworkTopology -import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString} +import org.mockito.ArgumentMatchers.{any, anyInt, anyString} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -70,47 +69,22 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) // Get the rack for a given host object FakeRackUtil { private val hostToRack = new mutable.HashMap[String, String]() - var loopCount = 0 + var numBatchInvocation = 0 def cleanUp() { hostToRack.clear() - loopCount = 0 + numBatchInvocation = 0 } def assignHostToRack(host: String, rack: String) { hostToRack(host) = rack } - def getRackForHost(host: String): Option[String] = { - loopCount = simulateRunResolveCommand(Seq(host)) - hostToRack.get(host) - } - def getRacksForHosts(hosts: List[String]): List[Option[String]] = { - loopCount = simulateRunResolveCommand(hosts) - hosts.map(hostToRack.get) - } - - /** - * This is a simulation of building and executing the resolution command. - * Simulate function `runResolveCommand()` in [[org.apache.hadoop.net.ScriptBasedMapping]]. - * If Seq has 100 elements, it returns 4. If Seq has 1 elements, it returns 1. - * @param args a list of arguments - * @return script execution times - */ - private def simulateRunResolveCommand(args: Seq[String]): Int = { - val maxArgs = 30 // Simulate NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_DEFAULT - var numProcessed = 0 - var loopCount = 0 - while (numProcessed != args.size) { - var start = maxArgs * loopCount - numProcessed = start - while (numProcessed < (start + maxArgs) && numProcessed < args.size) { - numProcessed += 1 - } - loopCount += 1 + if (hosts.nonEmpty && hosts.length != 1) { + numBatchInvocation += 1 } - loopCount + hosts.map(hostToRack.get) } } @@ -131,8 +105,6 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val executors = new mutable.HashMap[String, String] - val skipRackResolving = sc.conf.get(config.LOCALITY_WAIT_RACK) == 0L - for ((execId, host) <- liveExecutors) { addExecutor(execId, host) } @@ -178,20 +150,11 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex } } - override def getRackForHost(value: String): Option[String] = - if (skipRackResolving) { - Option(NetworkTopology.DEFAULT_RACK) - } else { - FakeRackUtil.getRackForHost(value) - } - + override def defaultRackValue: Option[String] = Option("None") - override def getRacksForHosts(values: List[String]): List[Option[String]] = - if (skipRackResolving) { - values.map(_ => Option(NetworkTopology.DEFAULT_RACK)) - } else { - FakeRackUtil.getRacksForHosts(values) - } + override def doGetRacksForHosts(values: List[String]): List[Option[String]] = { + FakeRackUtil.getRacksForHosts(values) + } } /** @@ -1671,7 +1634,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } assert(sched.skipRackResolving === false) assert(total === 100) // verify the total number not changed with SPARK-27038 - assert(FakeRackUtil.loopCount === 4) // verify script execution loop count decreased + assert(FakeRackUtil.numBatchInvocation === 1) } test("SPARK-27038 Verify the rack resolving time when spark.locality.wait is zero") { @@ -1691,7 +1654,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) assert(sched.skipRackResolving === true) // verify the total number not changed with SPARK-27038 - assert(manager.getPendingTasksForRack(NetworkTopology.DEFAULT_RACK).length === 100) - assert(FakeRackUtil.loopCount === 0) // verify no invocation of FakeRackUtil.getRacksForHosts() + assert(manager.getPendingTasksForRack(sched.defaultRackValue.get).length === 100) + assert(FakeRackUtil.numBatchInvocation === 0) } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index f16e35ef25753..a0f54120a6486 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -31,7 +31,7 @@ import org.apache.log4j.{Level, Logger} import org.apache.spark.internal.Logging /** - * Wrapper around YARN's [[RackResolver]]. This allows Spark tests to easily override the + * Re-implement YARN's [[RackResolver]]. This allows Spark tests to easily override the * default behavior, since YARN's class self-initializes the first time it's called, and * future calls all use the initial configuration. */ @@ -43,7 +43,7 @@ private[spark] class SparkRackResolver { } def resolve(conf: Configuration, hostName: String): String = { - RackResolver.resolve(conf, hostName).getNetworkLocation() + SparkRackResolver.coreResolve(conf, List(hostName)).head.getNetworkLocation } /** @@ -70,19 +70,12 @@ object SparkRackResolver extends Logging { private var dnsToSwitchMapping: DNSToSwitchMapping = _ private var initCalled = false - private def init(conf: Configuration): Unit = { + def coreResolve(conf: Configuration, hostNames: List[String]): List[Node] = { if (!initCalled) { initCalled = true val dnsToSwitchMappingClass = conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, classOf[ScriptBasedMapping], classOf[DNSToSwitchMapping]) - if (classOf[ScriptBasedMapping].isAssignableFrom(dnsToSwitchMappingClass)) { - val numArgs = conf.getInt(CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY, - CommonConfigurationKeysPublic.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_DEFAULT) - logInfo(s"Setting spark.hadoop.net.topology.script.number.args with a higher value " + - s"may reduce the time of rack resolving when submits a stage with a mass of tasks. " + - s"Current number is $numArgs") - } val newInstance = ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) .asInstanceOf[DNSToSwitchMapping] dnsToSwitchMapping = newInstance match { @@ -90,10 +83,7 @@ object SparkRackResolver extends Logging { case _ => new CachedDNSToSwitchMapping(newInstance) } } - } - def coreResolve(conf: Configuration, hostNames: List[String]): List[Node] = { - init(conf) val nodes = new ArrayBuffer[Node] val rNameList = dnsToSwitchMapping.resolve(hostNames.toList.asJava).asScala if (rNameList == null || rNameList.isEmpty) { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index 09e72d3771869..f246b011dced1 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -18,11 +18,9 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.net.NetworkTopology -import org.apache.hadoop.yarn.util.RackResolver import org.apache.spark._ import org.apache.spark.deploy.yarn.SparkRackResolver -import org.apache.spark.internal.config.LOCALITY_WAIT_RACK import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.Utils @@ -30,34 +28,18 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s private[spark] val resolver = new SparkRackResolver - // Add a on-off switch to save time for rack resolving - val skipRackResolving: Boolean = sc.conf.get(LOCALITY_WAIT_RACK) == 0L - - // By default, rack is unknown - override def getRackForHost(hostPort: String): Option[String] = { - if (skipRackResolving) { - Option(NetworkTopology.DEFAULT_RACK) - } else { - val host = Utils.parseHostPort(hostPort)._1 - Option(resolver.resolve(sc.hadoopConfiguration, host)) - } - } + override def defaultRackValue: Option[String] = Option(NetworkTopology.DEFAULT_RACK) /** * Get racks info for a list of host. - * Use [[SparkRackResolver]] to resolve racks instead of [[RackResolver]] - * before the dependency Hadoop applied YARN-9332 - * @param hostPorts host list to resolve - * @return rack list + * Use [[SparkRackResolver]] to resolve racks before the dependency Hadoop applied YARN-9332 + * @param hostPorts a list of task preferred locations to be resolved. + * @return resolved rack list. */ override def getRacksForHosts(hostPorts: List[String]): List[Option[String]] = { val hosts = hostPorts.map(Utils.parseHostPort(_)._1) - if (skipRackResolving) { - hosts.map(_ => Option(NetworkTopology.DEFAULT_RACK)) - } else { - resolver.resolve(sc.hadoopConfiguration, hosts).map { node => - Option(node.getNetworkLocation) - } + resolver.resolve(sc.hadoopConfiguration, hosts).map { node => + Option(node.getNetworkLocation) } } } From 7e4c729f1136d69d442dcb556031a41c037af81b Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Mon, 18 Mar 2019 16:18:09 +0800 Subject: [PATCH 09/28] update to lazy val to avoid mass invocations --- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index c8743f9233c20..d9b4b0c0c63dc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -812,7 +812,7 @@ private[spark] class TaskSchedulerImpl( } // Add a on-off switch to save time for rack resolving - def skipRackResolving: Boolean = sc.conf.get(LOCALITY_WAIT_RACK) == 0L + lazy val skipRackResolving: Boolean = sc.conf.get(LOCALITY_WAIT_RACK) == 0L /** * Rack is unknown by default. diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 759757bf7464f..8ffb4b3d118a4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -150,7 +150,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex } } - override def defaultRackValue: Option[String] = Option("None") + override def defaultRackValue: Option[String] = Option("default") override def doGetRacksForHosts(values: List[String]): List[Option[String]] = { FakeRackUtil.getRacksForHosts(values) From d4a7cde8b3bd49554c6ae7f82d464c0cbeb162a4 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Mon, 18 Mar 2019 16:25:43 +0800 Subject: [PATCH 10/28] some fix --- .../org/apache/spark/scheduler/cluster/YarnScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index f246b011dced1..6b363ffb12bbe 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -36,7 +36,7 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s * @param hostPorts a list of task preferred locations to be resolved. * @return resolved rack list. */ - override def getRacksForHosts(hostPorts: List[String]): List[Option[String]] = { + override def doGetRacksForHosts(hostPorts: List[String]): List[Option[String]] = { val hosts = hostPorts.map(Utils.parseHostPort(_)._1) resolver.resolve(sc.hadoopConfiguration, hosts).map { node => Option(node.getNetworkLocation) From d3e15922b133052ea279cfbf32e18513d08ab458 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Mon, 18 Mar 2019 21:05:13 +0800 Subject: [PATCH 11/28] De-duping the hosts to reduce this invocation further --- .../spark/scheduler/TaskSetManager.scala | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 31fbd86f2703b..b05b351889045 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -190,14 +190,15 @@ private[spark] class TaskSetManager( private def addPendingTasks(): Unit = { val (_, duration) = Utils.timeTakenMs { - val array = new ArrayBuffer[(String, Int)]() // Array(preferredLocation, task index) + val hostToIndices = new HashMap[String, ArrayBuffer[Int]]() for (i <- (0 until numTasks).reverse) { - addPendingTask(i, Some(array)) + addPendingTask(i, Option(hostToIndices)) } - // Convert preferred locations to racks in one invocation and zip with the origin indices - sched.getRacksForHosts(array.map(_._1).toList).zip(array.map(_._2)) foreach { - case (Some(rack), index) => - pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index + // Convert preferred locations to racks in one invocation and zip with the origin indices. + // We de-duping the hosts to reduce this invocation further. + sched.getRacksForHosts(hostToIndices.keySet.toList).zip(hostToIndices.values) foreach { + case (Some(rack), indices) => + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) ++= indices case _ => } } @@ -229,7 +230,7 @@ private[spark] class TaskSetManager( /** Add a task to all the pending-task lists that it should be on. */ private[spark] def addPendingTask(index: Int, - initializingTaskArray: Option[ArrayBuffer[(String, Int)]] = None) { + initializingHostToIndices: Option[HashMap[String, ArrayBuffer[Int]]] = None) { for (loc <- tasks(index).preferredLocations) { loc match { case e: ExecutorCacheTaskLocation => @@ -250,8 +251,10 @@ private[spark] class TaskSetManager( } pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer) += index - initializingTaskArray match { - case Some(array) => array += ((loc.host, index)) + initializingHostToIndices match { + case Some(hostToIndices) => + // when TaskSetManager initializing, preferredLocation -> task indices + hostToIndices.getOrElseUpdate(loc.host, new ArrayBuffer) += index case None => for (rack <- sched.getRackForHost(loc.host)) { pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index From 47add6fab684691ea3e7dbeb4a7c4399dfca28b7 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Tue, 19 Mar 2019 17:00:11 +0800 Subject: [PATCH 12/28] address many comments --- .../spark/internal/config/package.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 39 +++++++++++-------- .../spark/scheduler/TaskSetManager.scala | 39 +++++++++---------- .../spark/scheduler/TaskSetManagerSuite.scala | 33 +++++++--------- .../spark/deploy/yarn/SparkRackResolver.scala | 6 +-- .../scheduler/cluster/YarnScheduler.scala | 10 ++--- 6 files changed, 62 insertions(+), 67 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index ac9ef44d67554..3ef6cba8193f6 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1129,7 +1129,7 @@ package object config { private[spark] val LOCALITY_WAIT_RACK = ConfigBuilder("spark.locality.wait.rack") .fallbackConf(LOCALITY_WAIT) - private[spark] val REDUCER_MAX_SIZE_IN_FLIGHT = ConfigBuilder("spark.reducer.maxSizeInFlight") + private[spark] val REDUCER_MAX_SIZE_IN_FLIGHT = ConfigBuilder("spark.reducer.maxSizeInFlight") .doc("Maximum size of map outputs to fetch simultaneously from each reduce task, " + "in MiB unless otherwise specified. Since each output requires us to create a " + "buffer to receive it, this represents a fixed memory overhead per reduce task, " + diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d9b4b0c0c63dc..630a7c4495f49 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -375,9 +375,10 @@ private[spark] class TaskSchedulerImpl( executorIdToRunningTaskIds(o.executorId) = HashSet[Long]() newExecAvail = true } - for (rack <- getRackForHost(o.host)) { - hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += o.host - } + } + val hosts = offers.map(_.host) + for ((host, rack) <- hosts.zip(getRacksForHosts(hosts))) { + hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += host } // Before making any offers, remove any nodes from the blacklist whose blacklist has expired. Do @@ -764,7 +765,8 @@ private[spark] class TaskSchedulerImpl( execs -= executorId if (execs.isEmpty) { hostToExecutors -= host - for (rack <- getRackForHost(host); hosts <- hostsByRack.get(rack)) { + val rack = getRackForHost(host) + for (hosts <- hostsByRack.get(rack)) { hosts -= host if (hosts.isEmpty) { hostsByRack -= rack @@ -812,33 +814,36 @@ private[spark] class TaskSchedulerImpl( } // Add a on-off switch to save time for rack resolving - lazy val skipRackResolving: Boolean = sc.conf.get(LOCALITY_WAIT_RACK) == 0L + private def skipRackResolving: Boolean = sc.conf.get(LOCALITY_WAIT_RACK) == 0L /** - * Rack is unknown by default. + * Rack is `unknown` by default. * It can be override in different TaskScheduler, like Yarn. */ - def defaultRackValue: Option[String] = None + protected val defaultRackValue: String = "unknown" - def doGetRacksForHosts(preferredLocation: List[String]): List[Option[String]] = Nil + /** + * Get racks info for a host list. This is the internal method of [[getRacksForHosts]]. + * It should be override in different TaskScheduler. Return [[Nil]] by default. + */ + protected def doGetRacksForHosts(hosts: Seq[String]): Seq[String] = Nil - def getRackForHost(value: String): Option[String] = { + def getRackForHost(hosts: String): String = { if (skipRackResolving) { defaultRackValue } else { - doGetRacksForHosts(List(value)) match { - case Nil => None - case h :: Nil => h - case _ => None - } + doGetRacksForHosts(Seq(hosts)).head } } - def getRacksForHosts(values: List[String]): List[Option[String]] = { + /** + * null in return sequences will be replaced to [[defaultRackValue]]. + */ + def getRacksForHosts(hosts: Seq[String]): Seq[String] = { if (skipRackResolving) { - values.map(_ => defaultRackValue) + hosts.map(_ => defaultRackValue) } else { - doGetRacksForHosts(values) + doGetRacksForHosts(hosts).map(Option(_).getOrElse(defaultRackValue)) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index b05b351889045..fe021e29de40e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -192,17 +192,17 @@ private[spark] class TaskSetManager( val (_, duration) = Utils.timeTakenMs { val hostToIndices = new HashMap[String, ArrayBuffer[Int]]() for (i <- (0 until numTasks).reverse) { - addPendingTask(i, Option(hostToIndices)) + addPendingTask(i, Some(hostToIndices)) } // Convert preferred locations to racks in one invocation and zip with the origin indices. // We de-duping the hosts to reduce this invocation further. - sched.getRacksForHosts(hostToIndices.keySet.toList).zip(hostToIndices.values) foreach { - case (Some(rack), indices) => - pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) ++= indices - case _ => + for ( + (rack, indices) <- sched.getRacksForHosts(hostToIndices.keySet.toSeq) + .zip(hostToIndices.values)) { + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) ++= indices } } - logInfo(s"Adding pending tasks take $duration ms") + logDebug(s"Adding pending tasks take $duration ms") } /** @@ -229,7 +229,8 @@ private[spark] class TaskSetManager( private[scheduler] var emittedTaskSizeWarning = false /** Add a task to all the pending-task lists that it should be on. */ - private[spark] def addPendingTask(index: Int, + private[spark] def addPendingTask( + index: Int, initializingHostToIndices: Option[HashMap[String, ArrayBuffer[Int]]] = None) { for (loc <- tasks(index).preferredLocations) { loc match { @@ -256,9 +257,8 @@ private[spark] class TaskSetManager( // when TaskSetManager initializing, preferredLocation -> task indices hostToIndices.getOrElseUpdate(loc.host, new ArrayBuffer) += index case None => - for (rack <- sched.getRackForHost(loc.host)) { - pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index - } + pendingTasksForRack.getOrElseUpdate( + sched.getRackForHost(loc.host), new ArrayBuffer) += index } } @@ -388,13 +388,12 @@ private[spark] class TaskSetManager( // Check for rack-local tasks if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { - for (rack <- sched.getRackForHost(host)) { - for (index <- speculatableTasks if canRunOnHost(index)) { - val racks = tasks(index).preferredLocations.map(_.host).flatMap(sched.getRackForHost) - if (racks.contains(rack)) { - speculatableTasks -= index - return Some((index, TaskLocality.RACK_LOCAL)) - } + val rack = sched.getRackForHost(host) + for (index <- speculatableTasks if canRunOnHost(index)) { + val racks = tasks(index).preferredLocations.map(_.host).map(sched.getRackForHost) + if (racks.contains(rack)) { + speculatableTasks -= index + return Some((index, TaskLocality.RACK_LOCAL)) } } } @@ -438,10 +437,8 @@ private[spark] class TaskSetManager( } if (TaskLocality.isAllowed(maxLocality, TaskLocality.RACK_LOCAL)) { - for { - rack <- sched.getRackForHost(host) - index <- dequeueTaskFromList(execId, host, getPendingTasksForRack(rack)) - } { + val rack = sched.getRackForHost(host) + for (index <- dequeueTaskFromList(execId, host, getPendingTasksForRack(rack))) { return Some((index, TaskLocality.RACK_LOCAL, false)) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 8ffb4b3d118a4..5752f2858f118 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -80,11 +80,12 @@ object FakeRackUtil { hostToRack(host) = rack } - def getRacksForHosts(hosts: List[String]): List[Option[String]] = { + def getRacksForHosts(hosts: Seq[String]): Seq[String] = { + assert(hosts.toSet.size == hosts.size) // no dups in hosts if (hosts.nonEmpty && hosts.length != 1) { numBatchInvocation += 1 } - hosts.map(hostToRack.get) + hosts.map(hostToRack.getOrElse(_, null)) } } @@ -109,8 +110,8 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex addExecutor(execId, host) } - for ((execId, host) <- liveExecutors; rack <- getRackForHost(host)) { - hostsByRack.getOrElseUpdate(rack, new mutable.HashSet[String]()) += host + for ((execId, host) <- liveExecutors) { + hostsByRack.getOrElseUpdate(getRackForHost(host), new mutable.HashSet[String]()) += host } dagScheduler = new FakeDAGScheduler(sc, this) @@ -122,7 +123,8 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val hostId = host.get val executorsOnHost = hostToExecutors(hostId) executorsOnHost -= execId - for (rack <- getRackForHost(hostId); hosts <- hostsByRack.get(rack)) { + val rack = getRackForHost(hostId) + for (hosts <- hostsByRack.get(rack)) { hosts -= hostId if (hosts.isEmpty) { hostsByRack -= rack @@ -145,15 +147,13 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val executorsOnHost = hostToExecutors.getOrElseUpdate(host, new mutable.HashSet[String]) executorsOnHost += execId executorIdToHost += execId -> host - for (rack <- getRackForHost(host)) { - hostsByRack.getOrElseUpdate(rack, new mutable.HashSet[String]()) += host - } + hostsByRack.getOrElseUpdate(getRackForHost(host), new mutable.HashSet[String]()) += host } - override def defaultRackValue: Option[String] = Option("default") + override val defaultRackValue: String = "default" - override def doGetRacksForHosts(values: List[String]): List[Option[String]] = { - FakeRackUtil.getRacksForHosts(values) + override def doGetRacksForHosts(hosts: Seq[String]): Seq[String] = { + FakeRackUtil.getRacksForHosts(hosts) } } @@ -1612,7 +1612,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg result.accumUpdates, info3) } - test("SPARK-27038 Verify the rack resolving time has been reduced") { + test("SPARK-27038 Rack Resolution is done with a batch of de-duped hosts") { sc = new SparkContext("local", "test") for (i <- 0 to 99) { FakeRackUtil.assignHostToRack("host" + i, "rack" + (i % 20)) @@ -1626,18 +1626,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(100, locations: _*) val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - var total = 0 for (i <- 0 until 20) { val numTaskInRack = manager.getPendingTasksForRack("rack" + i).length assert(numTaskInRack === 5) // check rack assignment is still done correctly - total += numTaskInRack } - assert(sched.skipRackResolving === false) - assert(total === 100) // verify the total number not changed with SPARK-27038 assert(FakeRackUtil.numBatchInvocation === 1) } - test("SPARK-27038 Verify the rack resolving time when spark.locality.wait is zero") { + test("SPARK-27038 Rack resolving is skipped when spark.locality.wait is zero") { val conf = new SparkConf().set(config.LOCALITY_WAIT.key, "0") sc = new SparkContext("local", "test", conf) for (i <- 0 to 99) { @@ -1652,9 +1648,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(100, locations: _*) val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - assert(sched.skipRackResolving === true) // verify the total number not changed with SPARK-27038 - assert(manager.getPendingTasksForRack(sched.defaultRackValue.get).length === 100) + assert(manager.getPendingTasksForRack(sched.defaultRackValue).length === 100) assert(FakeRackUtil.numBatchInvocation === 0) } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index a0f54120a6486..b6377f027a6bc 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -43,7 +43,7 @@ private[spark] class SparkRackResolver { } def resolve(conf: Configuration, hostName: String): String = { - SparkRackResolver.coreResolve(conf, List(hostName)).head.getNetworkLocation + SparkRackResolver.coreResolve(conf, Seq(hostName)).head.getNetworkLocation } /** @@ -51,7 +51,7 @@ private[spark] class SparkRackResolver { * This should be changed to `RackResolver.resolve(conf, hostNames)` * in hadoop releases with YARN-9332. */ - def resolve(conf: Configuration, hostNames: List[String]): List[Node] = { + def resolve(conf: Configuration, hostNames: Seq[String]): Seq[Node] = { SparkRackResolver.coreResolve(conf, hostNames) } } @@ -70,7 +70,7 @@ object SparkRackResolver extends Logging { private var dnsToSwitchMapping: DNSToSwitchMapping = _ private var initCalled = false - def coreResolve(conf: Configuration, hostNames: List[String]): List[Node] = { + def coreResolve(conf: Configuration, hostNames: Seq[String]): Seq[Node] = { if (!initCalled) { initCalled = true val dnsToSwitchMappingClass = diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index 6b363ffb12bbe..99daa33e01849 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -28,18 +28,16 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s private[spark] val resolver = new SparkRackResolver - override def defaultRackValue: Option[String] = Option(NetworkTopology.DEFAULT_RACK) + override val defaultRackValue: String = NetworkTopology.DEFAULT_RACK /** - * Get racks info for a list of host. + * Get racks info for a host list. * Use [[SparkRackResolver]] to resolve racks before the dependency Hadoop applied YARN-9332 * @param hostPorts a list of task preferred locations to be resolved. * @return resolved rack list. */ - override def doGetRacksForHosts(hostPorts: List[String]): List[Option[String]] = { + override def doGetRacksForHosts(hostPorts: Seq[String]): Seq[String] = { val hosts = hostPorts.map(Utils.parseHostPort(_)._1) - resolver.resolve(sc.hadoopConfiguration, hosts).map { node => - Option(node.getNetworkLocation) - } + resolver.resolve(sc.hadoopConfiguration, hosts).map(_.getNetworkLocation) } } From e8ab99e2f0f23498254d2fd21873f6f87367262b Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Tue, 19 Mar 2019 17:50:17 +0800 Subject: [PATCH 13/28] address comment --- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index fe021e29de40e..abeaa682d63d3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -194,8 +194,8 @@ private[spark] class TaskSetManager( for (i <- (0 until numTasks).reverse) { addPendingTask(i, Some(hostToIndices)) } - // Convert preferred locations to racks in one invocation and zip with the origin indices. - // We de-duping the hosts to reduce this invocation further. + // Resolve the rack for each host. This can be somehow slow, so de-dupe the list of hosts, + // and assign the rack to all relevant task indices. for ( (rack, indices) <- sched.getRacksForHosts(hostToIndices.keySet.toSeq) .zip(hostToIndices.values)) { From 11fdd41c47bc369e932a346e1f00b83c443b6943 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Tue, 19 Mar 2019 17:55:04 +0800 Subject: [PATCH 14/28] thread-safe SparkRackResolver --- .../spark/deploy/yarn/SparkRackResolver.scala | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index b6377f027a6bc..61d0aa33f6c0e 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -68,23 +68,26 @@ private[spark] class SparkRackResolver { */ object SparkRackResolver extends Logging { private var dnsToSwitchMapping: DNSToSwitchMapping = _ - private var initCalled = false + @volatile private var initCalled = false def coreResolve(conf: Configuration, hostNames: Seq[String]): Seq[Node] = { if (!initCalled) { - initCalled = true - val dnsToSwitchMappingClass = - conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, - classOf[ScriptBasedMapping], classOf[DNSToSwitchMapping]) - val newInstance = ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) - .asInstanceOf[DNSToSwitchMapping] - dnsToSwitchMapping = newInstance match { - case _: CachedDNSToSwitchMapping => newInstance - case _ => new CachedDNSToSwitchMapping(newInstance) + synchronized { + initCalled = true + val dnsToSwitchMappingClass = + conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + classOf[ScriptBasedMapping], classOf[DNSToSwitchMapping]) + val newInstance = ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) + .asInstanceOf[DNSToSwitchMapping] + dnsToSwitchMapping = newInstance match { + case _: CachedDNSToSwitchMapping => newInstance + case _ => new CachedDNSToSwitchMapping(newInstance) + } } } val nodes = new ArrayBuffer[Node] + // dnsToSwitchMapping is thread-safe val rNameList = dnsToSwitchMapping.resolve(hostNames.toList.asJava).asScala if (rNameList == null || rNameList.isEmpty) { hostNames.foreach(nodes += new NodeBase(_, NetworkTopology.DEFAULT_RACK)) From ade4caa96e846fefc017b39b51c9d76298636051 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Tue, 19 Mar 2019 18:10:52 +0800 Subject: [PATCH 15/28] fix the ut to address comment --- .../spark/scheduler/TaskSetManagerSuite.scala | 29 ++++++++++++++----- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 5752f2858f118..9c96e33cbce0c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -1613,12 +1613,17 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } test("SPARK-27038 Rack Resolution is done with a batch of de-duped hosts") { - sc = new SparkContext("local", "test") - for (i <- 0 to 99) { + val conf = new SparkConf() + .set(config.LOCALITY_WAIT.key, "0") + .set(config.LOCALITY_WAIT_RACK.key, "1s") + sc = new SparkContext("local", "test", conf) + // Create a cluster with 20 racks, with hosts spread out among them + val execAndHost = (0 to 199).map { i => FakeRackUtil.assignHostToRack("host" + i, "rack" + (i % 20)) + ("exec" + i, "host" + i) } - sched = new FakeTaskScheduler(sc, - ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + sched = new FakeTaskScheduler(sc, execAndHost: _*) + // make a taskset with preferred locations on the first 100 hosts in our cluster val locations = new ArrayBuffer[Seq[TaskLocation]]() for (i <- 0 to 99) { locations += Seq(TaskLocation("host" + i)) @@ -1626,9 +1631,19 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(100, locations: _*) val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - for (i <- 0 until 20) { - val numTaskInRack = manager.getPendingTasksForRack("rack" + i).length - assert(numTaskInRack === 5) // check rack assignment is still done correctly + // with rack locality, reject an offer on a host with an unknown rack + assert(manager.resourceOffer("otherExec", "otherHost", TaskLocality.RACK_LOCAL).isEmpty) + (0 until 20).foreach { rackIdx => + (0 until 5).foreach { offerIdx => + // if we offer hosts which are not in preferred locations, + // we'll reject them at NODE_LOCAL level, + // but accept them at RACK_LOCAL level if they're on OK racks + val hostIdx = 100 + rackIdx + assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.NODE_LOCAL) + .isEmpty) + assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.RACK_LOCAL) + .isDefined) + } } assert(FakeRackUtil.numBatchInvocation === 1) } From c9dace8b0d12ba068e575e30fdd27bc7cbbd7dcc Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Tue, 19 Mar 2019 18:42:25 +0800 Subject: [PATCH 16/28] use batch resolve in LocalityPreferredContainerPlacementStrategy --- .../yarn/LocalityPreferredContainerPlacementStrategy.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala index 0a7a16f468fbd..8cd5c1292d1ba 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala @@ -138,9 +138,8 @@ private[yarn] class LocalityPreferredContainerPlacementStrategy( // Only filter out the ratio which is larger than 0, which means the current host can // still be allocated with new container request. val hosts = preferredLocalityRatio.filter(_._2 > 0).keys.toArray - val racks = hosts.map { h => - resolver.resolve(yarnConf, h) - }.toSet + val racks = resolver.resolve(yarnConf, hosts).map(_.getNetworkLocation) + .filter(_ != null).toSet containerLocalityPreferences += ContainerLocalityPreferences(hosts, racks.toArray) // Minus 1 each time when the host is used. When the current ratio is 0, From e2faee6b0680cab7a7dd50d809126f4214849601 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Wed, 20 Mar 2019 13:24:28 +0800 Subject: [PATCH 17/28] double-checked locking --- .../spark/deploy/yarn/SparkRackResolver.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index 61d0aa33f6c0e..efd05bdab432d 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -67,21 +67,21 @@ private[spark] class SparkRackResolver { * In the meantime, this is a re-implementation for spark's use. */ object SparkRackResolver extends Logging { - private var dnsToSwitchMapping: DNSToSwitchMapping = _ - @volatile private var initCalled = false + @volatile private var dnsToSwitchMapping: DNSToSwitchMapping = _ def coreResolve(conf: Configuration, hostNames: Seq[String]): Seq[Node] = { - if (!initCalled) { + if (dnsToSwitchMapping == null) { synchronized { - initCalled = true - val dnsToSwitchMappingClass = - conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, - classOf[ScriptBasedMapping], classOf[DNSToSwitchMapping]) - val newInstance = ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) - .asInstanceOf[DNSToSwitchMapping] - dnsToSwitchMapping = newInstance match { - case _: CachedDNSToSwitchMapping => newInstance - case _ => new CachedDNSToSwitchMapping(newInstance) + if (dnsToSwitchMapping == null) { + val dnsToSwitchMappingClass = + conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + classOf[ScriptBasedMapping], classOf[DNSToSwitchMapping]) + val newInstance = ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) + .asInstanceOf[DNSToSwitchMapping] + dnsToSwitchMapping = newInstance match { + case _: CachedDNSToSwitchMapping => newInstance + case _ => new CachedDNSToSwitchMapping(newInstance) + } } } } From aca97f1005f8cebd9e217b2bddacc8f3f776a50b Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Wed, 20 Mar 2019 15:36:50 +0800 Subject: [PATCH 18/28] handle Nil --- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 630a7c4495f49..6a062642764ff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -832,7 +832,10 @@ private[spark] class TaskSchedulerImpl( if (skipRackResolving) { defaultRackValue } else { - doGetRacksForHosts(Seq(hosts)).head + doGetRacksForHosts(Seq(hosts)) match { + case Nil => defaultRackValue + case head :: Nil => head + } } } From 75d22edf2de4afc403fd36b97fa32462a2e8b50a Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Thu, 21 Mar 2019 15:08:50 +0800 Subject: [PATCH 19/28] address comments --- .../spark/scheduler/TaskSchedulerImpl.scala | 12 +++++++---- .../spark/scheduler/TaskSetManager.scala | 20 +++++++++---------- .../spark/scheduler/TaskSetManagerSuite.scala | 6 +++--- 3 files changed, 21 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 6a062642764ff..dd03cc9d7c51a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -376,7 +376,7 @@ private[spark] class TaskSchedulerImpl( newExecAvail = true } } - val hosts = offers.map(_.host) + val hosts = offers.map(_.host).toSet.toSeq for ((host, rack) <- hosts.zip(getRacksForHosts(hosts))) { hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += host } @@ -823,10 +823,14 @@ private[spark] class TaskSchedulerImpl( protected val defaultRackValue: String = "unknown" /** - * Get racks info for a host list. This is the internal method of [[getRacksForHosts]]. - * It should be override in different TaskScheduler. Return [[Nil]] by default. + * Get racks info for hosts. This is the internal method of [[getRacksForHosts]]. + * It should be override in different TaskScheduler. + * The return racks must have to be the same length as the hosts passed in. + * Return [[defaultRackValue]] sequence by default. */ - protected def doGetRacksForHosts(hosts: Seq[String]): Seq[String] = Nil + protected def doGetRacksForHosts(hosts: Seq[String]): Seq[String] = { + hosts.map(_ => defaultRackValue) + } def getRackForHost(hosts: String): String = { if (skipRackResolving) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index abeaa682d63d3..115e4a42d4fad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -192,9 +192,9 @@ private[spark] class TaskSetManager( val (_, duration) = Utils.timeTakenMs { val hostToIndices = new HashMap[String, ArrayBuffer[Int]]() for (i <- (0 until numTasks).reverse) { - addPendingTask(i, Some(hostToIndices)) + addPendingTask(i, initializing = true, Some(hostToIndices)) } - // Resolve the rack for each host. This can be somehow slow, so de-dupe the list of hosts, + // Resolve the rack for each host. This can be slow, so de-dupe the list of hosts, // and assign the rack to all relevant task indices. for ( (rack, indices) <- sched.getRacksForHosts(hostToIndices.keySet.toSeq) @@ -231,7 +231,8 @@ private[spark] class TaskSetManager( /** Add a task to all the pending-task lists that it should be on. */ private[spark] def addPendingTask( index: Int, - initializingHostToIndices: Option[HashMap[String, ArrayBuffer[Int]]] = None) { + initializing: Boolean = false, + initializingMap: Option[HashMap[String, ArrayBuffer[Int]]] = None) { for (loc <- tasks(index).preferredLocations) { loc match { case e: ExecutorCacheTaskLocation => @@ -252,13 +253,12 @@ private[spark] class TaskSetManager( } pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer) += index - initializingHostToIndices match { - case Some(hostToIndices) => - // when TaskSetManager initializing, preferredLocation -> task indices - hostToIndices.getOrElseUpdate(loc.host, new ArrayBuffer) += index - case None => - pendingTasksForRack.getOrElseUpdate( - sched.getRackForHost(loc.host), new ArrayBuffer) += index + if (initializing) { + // preferredLocation -> task indices, initializingMap used when TaskSetManager initializing + initializingMap.foreach(_.getOrElseUpdate(loc.host, new ArrayBuffer) += index) + } else { + pendingTasksForRack.getOrElseUpdate( + sched.getRackForHost(loc.host), new ArrayBuffer) += index } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 9c96e33cbce0c..dfb3094e0fa54 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,7 +22,7 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.mockito.ArgumentMatchers.{any, anyInt, anyString} +import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -1325,7 +1325,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY) // Assert the task has been black listed on the executor it was last executed on. - when(taskSetManagerSpy.addPendingTask(anyInt(), any())).thenAnswer( + when(taskSetManagerSpy.addPendingTask(anyInt(), anyBoolean(), any())).thenAnswer( new Answer[Unit] { override def answer(invocationOnMock: InvocationOnMock): Unit = { val task: Int = invocationOnMock.getArgument(0) @@ -1339,7 +1339,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val e = new ExceptionFailure("a", "b", Array(), "c", None) taskSetManagerSpy.handleFailedTask(taskDesc.get.taskId, TaskState.FAILED, e) - verify(taskSetManagerSpy, times(1)).addPendingTask(anyInt(), any()) + verify(taskSetManagerSpy, times(1)).addPendingTask(anyInt(), anyBoolean(), any()) } test("SPARK-21563 context's added jars shouldn't change mid-TaskSet") { From 011b4c0c67427ce048a1cc93de9549954c047989 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Thu, 21 Mar 2019 15:52:28 +0800 Subject: [PATCH 20/28] refactor SparkRackResolver --- .../spark/deploy/yarn/SparkRackResolver.scala | 93 +++++++++++-------- .../scheduler/cluster/YarnScheduler.scala | 2 +- 2 files changed, 56 insertions(+), 39 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index efd05bdab432d..cb61814eabf5d 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -35,57 +35,38 @@ import org.apache.spark.internal.Logging * default behavior, since YARN's class self-initializes the first time it's called, and * future calls all use the initial configuration. */ -private[spark] class SparkRackResolver { +private[spark] class SparkRackResolver(conf: Configuration) extends Logging { // RackResolver logs an INFO message whenever it resolves a rack, which is way too often. if (Logger.getLogger(classOf[RackResolver]).getLevel == null) { Logger.getLogger(classOf[RackResolver]).setLevel(Level.WARN) } + private val dnsToSwitchMapping: DNSToSwitchMapping = { + val dnsToSwitchMappingClass = + conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + classOf[ScriptBasedMapping], classOf[DNSToSwitchMapping]) + ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) + .asInstanceOf[DNSToSwitchMapping] match { + case c: CachedDNSToSwitchMapping => c + case o => new CachedDNSToSwitchMapping(o) + } + } + def resolve(conf: Configuration, hostName: String): String = { - SparkRackResolver.coreResolve(conf, Seq(hostName)).head.getNetworkLocation + SparkRackResolver(conf).coreResolve(Seq(hostName)).head.getNetworkLocation } /** - * Added in SPARK-27038. - * This should be changed to `RackResolver.resolve(conf, hostNames)` - * in hadoop releases with YARN-9332. - */ + * Added in SPARK-27038. + * This should be changed to `RackResolver.resolve(conf, hostNames)` + * in hadoop releases with YARN-9332. + */ def resolve(conf: Configuration, hostNames: Seq[String]): Seq[Node] = { - SparkRackResolver.coreResolve(conf, hostNames) + SparkRackResolver(conf).coreResolve(hostNames) } -} - -/** - * Utility to resolve the rack for hosts in an efficient manner. - * It will cache the rack for individual hosts to avoid - * repeatedly performing the same expensive lookup. - * - * Its logic refers [[org.apache.hadoop.yarn.util.RackResolver]] and enhanced. - * This will be unnecessary in hadoop releases with YARN-9332. - * With that, we could just directly use [[org.apache.hadoop.yarn.util.RackResolver]]. - * In the meantime, this is a re-implementation for spark's use. - */ -object SparkRackResolver extends Logging { - @volatile private var dnsToSwitchMapping: DNSToSwitchMapping = _ - - def coreResolve(conf: Configuration, hostNames: Seq[String]): Seq[Node] = { - if (dnsToSwitchMapping == null) { - synchronized { - if (dnsToSwitchMapping == null) { - val dnsToSwitchMappingClass = - conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, - classOf[ScriptBasedMapping], classOf[DNSToSwitchMapping]) - val newInstance = ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) - .asInstanceOf[DNSToSwitchMapping] - dnsToSwitchMapping = newInstance match { - case _: CachedDNSToSwitchMapping => newInstance - case _ => new CachedDNSToSwitchMapping(newInstance) - } - } - } - } + private def coreResolve(hostNames: Seq[String]): Seq[Node] = { val nodes = new ArrayBuffer[Node] // dnsToSwitchMapping is thread-safe val rNameList = dnsToSwitchMapping.resolve(hostNames.toList.asJava).asScala @@ -107,3 +88,39 @@ object SparkRackResolver extends Logging { nodes.toList } } + +/** + * Utility to resolve the rack for hosts in an efficient manner. + * It will cache the rack for individual hosts to avoid + * repeatedly performing the same expensive lookup. + * + * Its logic refers [[org.apache.hadoop.yarn.util.RackResolver]] and enhanced. + * This will be unnecessary in hadoop releases with YARN-9332. + * With that, we could just directly use [[org.apache.hadoop.yarn.util.RackResolver]]. + * In the meantime, this is a re-implementation for spark's use. + */ +object SparkRackResolver extends Logging { + private var instance: SparkRackResolver = _ + + /** + * It will return the static resolver instance. + * If you want to generate a separate one, please use [[get]] + * @param conf + * @return the static resolver instance + */ + def apply(conf: Configuration): SparkRackResolver = { + if (instance == null) { + synchronized { + if (instance == null) { + instance = new SparkRackResolver(conf) + } + } + } + instance + } + + /** + * Instantiate a separate resolver with a separate config. + */ + def get(conf: Configuration): SparkRackResolver = new SparkRackResolver(conf) +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index 99daa33e01849..63202941a260e 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.Utils private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(sc) { - private[spark] val resolver = new SparkRackResolver + private[spark] val resolver = SparkRackResolver(sc.hadoopConfiguration) override val defaultRackValue: String = NetworkTopology.DEFAULT_RACK From 2876ad35378723e283cb20140ed0cb1b6b09e3fe Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Thu, 21 Mar 2019 17:35:06 +0800 Subject: [PATCH 21/28] fix build failure --- .../org/apache/spark/scheduler/TaskSetManagerSuite.scala | 6 +++--- .../org/apache/spark/deploy/yarn/SparkRackResolver.scala | 8 ++++---- .../scala/org/apache/spark/deploy/yarn/YarnRMClient.scala | 2 +- .../deploy/yarn/LocalityPlacementStrategySuite.scala | 2 +- .../org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala | 4 ++-- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index dfb3094e0fa54..b3679cb54821e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -1612,7 +1612,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg result.accumUpdates, info3) } - test("SPARK-27038 Rack Resolution is done with a batch of de-duped hosts") { + test("SPARK-13704 Rack Resolution is done with a batch of de-duped hosts") { val conf = new SparkConf() .set(config.LOCALITY_WAIT.key, "0") .set(config.LOCALITY_WAIT_RACK.key, "1s") @@ -1648,7 +1648,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(FakeRackUtil.numBatchInvocation === 1) } - test("SPARK-27038 Rack resolving is skipped when spark.locality.wait is zero") { + test("SPARK-13704 Rack resolving is skipped when spark.locality.wait is zero") { val conf = new SparkConf().set(config.LOCALITY_WAIT.key, "0") sc = new SparkContext("local", "test", conf) for (i <- 0 to 99) { @@ -1663,7 +1663,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(100, locations: _*) val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - // verify the total number not changed with SPARK-27038 + // verify the total number not changed with SPARK-13704 assert(manager.getPendingTasksForRack(sched.defaultRackValue).length === 100) assert(FakeRackUtil.numBatchInvocation === 0) } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index cb61814eabf5d..92af87038a71b 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -58,10 +58,10 @@ private[spark] class SparkRackResolver(conf: Configuration) extends Logging { } /** - * Added in SPARK-27038. - * This should be changed to `RackResolver.resolve(conf, hostNames)` - * in hadoop releases with YARN-9332. - */ + * Added in SPARK-13704. + * This should be changed to `RackResolver.resolve(conf, hostNames)` + * in hadoop releases with YARN-9332. + */ def resolve(conf: Configuration, hostNames: Seq[String]): Seq[Node] = { SparkRackResolver(conf).coreResolve(hostNames) } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index cf16edf16c034..6eff7d9072667 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -83,7 +83,7 @@ private[spark] class YarnRMClient extends Logging { localResources: Map[String, LocalResource]): YarnAllocator = { require(registered, "Must register AM before creating allocator.") new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, appAttemptId, securityMgr, - localResources, new SparkRackResolver()) + localResources, new SparkRackResolver(conf)) } /** diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala index b7f25656e49ac..98806da7446f2 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala @@ -58,7 +58,7 @@ class LocalityPlacementStrategySuite extends SparkFunSuite { val resource = Resource.newInstance(8 * 1024, 4) val strategy = new LocalityPreferredContainerPlacementStrategy(new SparkConf(), - yarnConf, resource, new MockResolver()) + yarnConf, resource, new MockResolver(yarnConf)) val totalTasks = 32 * 1024 val totalContainers = totalTasks / 16 diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 53a538dc1de29..191649a608f6e 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -37,7 +37,7 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.SplitInfo import org.apache.spark.util.ManualClock -class MockResolver extends SparkRackResolver { +class MockResolver(conf: Configuration) extends SparkRackResolver(conf) { override def resolve(conf: Configuration, hostName: String): String = { if (hostName == "host3") "/rack2" else "/rack1" @@ -112,7 +112,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter appAttemptId, new SecurityManager(sparkConf), Map(), - new MockResolver(), + new MockResolver(conf), clock) } From 06a6264af43c291b8b62fc700d6afc496e94a681 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Fri, 22 Mar 2019 12:00:27 +0800 Subject: [PATCH 22/28] fix the ut and keep the original behavior of getRackForHost --- .../spark/scheduler/TaskSchedulerImpl.scala | 31 +++++++++---------- .../spark/scheduler/TaskSetManager.scala | 28 ++++++++++------- .../spark/scheduler/TaskSetManagerSuite.scala | 23 +++++++------- .../scheduler/cluster/YarnScheduler.scala | 10 +++--- .../yarn/LocalityPlacementStrategySuite.scala | 2 +- .../deploy/yarn/YarnAllocatorSuite.scala | 4 +-- 6 files changed, 51 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index dd03cc9d7c51a..d4ffaf490f060 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -156,6 +156,12 @@ private[spark] class TaskSchedulerImpl( private[scheduler] var barrierCoordinator: RpcEndpoint = null + /** + * It can be override in different TaskScheduler, like Yarn. + * None by default. This should be initialized before any invocation. + */ + protected val defaultRackValue: Option[String] = None + private def maybeInitBarrierCoordinator(): Unit = { if (barrierCoordinator == null) { barrierCoordinator = new BarrierCoordinator(barrierSyncTimeout, sc.listenerBus, @@ -377,7 +383,7 @@ private[spark] class TaskSchedulerImpl( } } val hosts = offers.map(_.host).toSet.toSeq - for ((host, rack) <- hosts.zip(getRacksForHosts(hosts))) { + for ((host, Some(rack)) <- hosts.zip(getRacksForHosts(hosts))) { hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += host } @@ -765,8 +771,7 @@ private[spark] class TaskSchedulerImpl( execs -= executorId if (execs.isEmpty) { hostToExecutors -= host - val rack = getRackForHost(host) - for (hosts <- hostsByRack.get(rack)) { + for (rack <- getRackForHost(host); hosts <- hostsByRack.get(rack)) { hosts -= host if (hosts.isEmpty) { hostsByRack -= rack @@ -816,41 +821,33 @@ private[spark] class TaskSchedulerImpl( // Add a on-off switch to save time for rack resolving private def skipRackResolving: Boolean = sc.conf.get(LOCALITY_WAIT_RACK) == 0L - /** - * Rack is `unknown` by default. - * It can be override in different TaskScheduler, like Yarn. - */ - protected val defaultRackValue: String = "unknown" - /** * Get racks info for hosts. This is the internal method of [[getRacksForHosts]]. * It should be override in different TaskScheduler. * The return racks must have to be the same length as the hosts passed in. * Return [[defaultRackValue]] sequence by default. */ - protected def doGetRacksForHosts(hosts: Seq[String]): Seq[String] = { + protected def doGetRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { hosts.map(_ => defaultRackValue) } - def getRackForHost(hosts: String): String = { + // By default, rack is unknown + def getRackForHost(hosts: String): Option[String] = { if (skipRackResolving) { defaultRackValue } else { - doGetRacksForHosts(Seq(hosts)) match { - case Nil => defaultRackValue - case head :: Nil => head - } + doGetRacksForHosts(Seq(hosts)).head } } /** * null in return sequences will be replaced to [[defaultRackValue]]. */ - def getRacksForHosts(hosts: Seq[String]): Seq[String] = { + def getRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { if (skipRackResolving) { hosts.map(_ => defaultRackValue) } else { - doGetRacksForHosts(hosts).map(Option(_).getOrElse(defaultRackValue)) + doGetRacksForHosts(hosts) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 115e4a42d4fad..96bcf9a979ce1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -197,7 +197,7 @@ private[spark] class TaskSetManager( // Resolve the rack for each host. This can be slow, so de-dupe the list of hosts, // and assign the rack to all relevant task indices. for ( - (rack, indices) <- sched.getRacksForHosts(hostToIndices.keySet.toSeq) + (Some(rack), indices) <- sched.getRacksForHosts(hostToIndices.keySet.toSeq) .zip(hostToIndices.values)) { pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) ++= indices } @@ -257,8 +257,9 @@ private[spark] class TaskSetManager( // preferredLocation -> task indices, initializingMap used when TaskSetManager initializing initializingMap.foreach(_.getOrElseUpdate(loc.host, new ArrayBuffer) += index) } else { - pendingTasksForRack.getOrElseUpdate( - sched.getRackForHost(loc.host), new ArrayBuffer) += index + for (rack <- sched.getRackForHost(loc.host)) { + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index + } } } @@ -357,7 +358,7 @@ private[spark] class TaskSetManager( val executors = prefs.flatMap(_ match { case e: ExecutorCacheTaskLocation => Some(e.executorId) case _ => None - }); + }) if (executors.contains(execId)) { speculatableTasks -= index return Some((index, TaskLocality.PROCESS_LOCAL)) @@ -388,12 +389,13 @@ private[spark] class TaskSetManager( // Check for rack-local tasks if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { - val rack = sched.getRackForHost(host) - for (index <- speculatableTasks if canRunOnHost(index)) { - val racks = tasks(index).preferredLocations.map(_.host).map(sched.getRackForHost) - if (racks.contains(rack)) { - speculatableTasks -= index - return Some((index, TaskLocality.RACK_LOCAL)) + for (rack <- sched.getRackForHost(host)) { + for (index <- speculatableTasks if canRunOnHost(index)) { + val racks = tasks(index).preferredLocations.map(_.host).map(sched.getRackForHost) + if (racks.contains(rack)) { + speculatableTasks -= index + return Some((index, TaskLocality.RACK_LOCAL)) + } } } } @@ -437,8 +439,10 @@ private[spark] class TaskSetManager( } if (TaskLocality.isAllowed(maxLocality, TaskLocality.RACK_LOCAL)) { - val rack = sched.getRackForHost(host) - for (index <- dequeueTaskFromList(execId, host, getPendingTasksForRack(rack))) { + for { + rack <- sched.getRackForHost(host) + index <- dequeueTaskFromList(execId, host, getPendingTasksForRack(rack)) + } { return Some((index, TaskLocality.RACK_LOCAL, false)) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index b3679cb54821e..2b428d498ac25 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -80,12 +80,12 @@ object FakeRackUtil { hostToRack(host) = rack } - def getRacksForHosts(hosts: Seq[String]): Seq[String] = { + def getRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { assert(hosts.toSet.size == hosts.size) // no dups in hosts if (hosts.nonEmpty && hosts.length != 1) { numBatchInvocation += 1 } - hosts.map(hostToRack.getOrElse(_, null)) + hosts.map(hostToRack.get(_)) } } @@ -106,12 +106,14 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val executors = new mutable.HashMap[String, String] + // this must be initialized before addExecutor + override val defaultRackValue: Option[String] = Some("default") for ((execId, host) <- liveExecutors) { addExecutor(execId, host) } - for ((execId, host) <- liveExecutors) { - hostsByRack.getOrElseUpdate(getRackForHost(host), new mutable.HashSet[String]()) += host + for ((execId, host) <- liveExecutors; rack <- getRackForHost(host)) { + hostsByRack.getOrElseUpdate(rack, new mutable.HashSet[String]()) += host } dagScheduler = new FakeDAGScheduler(sc, this) @@ -123,8 +125,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val hostId = host.get val executorsOnHost = hostToExecutors(hostId) executorsOnHost -= execId - val rack = getRackForHost(hostId) - for (hosts <- hostsByRack.get(rack)) { + for (rack <- getRackForHost(hostId); hosts <- hostsByRack.get(rack)) { hosts -= hostId if (hosts.isEmpty) { hostsByRack -= rack @@ -147,12 +148,12 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val executorsOnHost = hostToExecutors.getOrElseUpdate(host, new mutable.HashSet[String]) executorsOnHost += execId executorIdToHost += execId -> host - hostsByRack.getOrElseUpdate(getRackForHost(host), new mutable.HashSet[String]()) += host + for (rack <- getRackForHost(host)) { + hostsByRack.getOrElseUpdate(rack, new mutable.HashSet[String]()) += host + } } - override val defaultRackValue: String = "default" - - override def doGetRacksForHosts(hosts: Seq[String]): Seq[String] = { + override def doGetRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { FakeRackUtil.getRacksForHosts(hosts) } } @@ -1664,7 +1665,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // verify the total number not changed with SPARK-13704 - assert(manager.getPendingTasksForRack(sched.defaultRackValue).length === 100) + assert(manager.getPendingTasksForRack(sched.defaultRackValue.get).length === 100) assert(FakeRackUtil.numBatchInvocation === 0) } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index 63202941a260e..c67cb79975403 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -26,9 +26,9 @@ import org.apache.spark.util.Utils private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(sc) { - private[spark] val resolver = SparkRackResolver(sc.hadoopConfiguration) + override val defaultRackValue: Option[String] = Some(NetworkTopology.DEFAULT_RACK) - override val defaultRackValue: String = NetworkTopology.DEFAULT_RACK + private[spark] val resolver = SparkRackResolver(sc.hadoopConfiguration) /** * Get racks info for a host list. @@ -36,8 +36,10 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s * @param hostPorts a list of task preferred locations to be resolved. * @return resolved rack list. */ - override def doGetRacksForHosts(hostPorts: Seq[String]): Seq[String] = { + override def doGetRacksForHosts(hostPorts: Seq[String]): Seq[Option[String]] = { val hosts = hostPorts.map(Utils.parseHostPort(_)._1) - resolver.resolve(sc.hadoopConfiguration, hosts).map(_.getNetworkLocation) + resolver.resolve(sc.hadoopConfiguration, hosts).map { node => + Option(node.getNetworkLocation) + } } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala index 98806da7446f2..b7f25656e49ac 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala @@ -58,7 +58,7 @@ class LocalityPlacementStrategySuite extends SparkFunSuite { val resource = Resource.newInstance(8 * 1024, 4) val strategy = new LocalityPreferredContainerPlacementStrategy(new SparkConf(), - yarnConf, resource, new MockResolver(yarnConf)) + yarnConf, resource, new MockResolver()) val totalTasks = 32 * 1024 val totalContainers = totalTasks / 16 diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 191649a608f6e..a22c4eb9d61d5 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -37,7 +37,7 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.SplitInfo import org.apache.spark.util.ManualClock -class MockResolver(conf: Configuration) extends SparkRackResolver(conf) { +class MockResolver() extends SparkRackResolver(new Configuration()) { override def resolve(conf: Configuration, hostName: String): String = { if (hostName == "host3") "/rack2" else "/rack1" @@ -112,7 +112,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter appAttemptId, new SecurityManager(sparkConf), Map(), - new MockResolver(conf), + new MockResolver(), clock) } From 92ef335c27b4a0d74e5831da370a920eb90cc191 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Fri, 22 Mar 2019 12:16:41 +0800 Subject: [PATCH 23/28] trival fix --- .../main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 2 +- .../scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala | 2 +- .../org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala | 3 ++- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 96bcf9a979ce1..87aea9fda27f9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -391,7 +391,7 @@ private[spark] class TaskSetManager( if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { for (rack <- sched.getRackForHost(host)) { for (index <- speculatableTasks if canRunOnHost(index)) { - val racks = tasks(index).preferredLocations.map(_.host).map(sched.getRackForHost) + val racks = tasks(index).preferredLocations.map(_.host).flatMap(sched.getRackForHost) if (racks.contains(rack)) { speculatableTasks -= index return Some((index, TaskLocality.RACK_LOCAL)) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index 92af87038a71b..f09b6776a0a32 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -35,7 +35,7 @@ import org.apache.spark.internal.Logging * default behavior, since YARN's class self-initializes the first time it's called, and * future calls all use the initial configuration. */ -private[spark] class SparkRackResolver(conf: Configuration) extends Logging { +private[yarn] class SparkRackResolver(conf: Configuration) extends Logging { // RackResolver logs an INFO message whenever it resolves a rack, which is way too often. if (Logger.getLogger(classOf[RackResolver]).getLevel == null) { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index a22c4eb9d61d5..edf380ea7b00b 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -31,13 +31,14 @@ import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.SplitInfo import org.apache.spark.util.ManualClock -class MockResolver() extends SparkRackResolver(new Configuration()) { +class MockResolver() extends SparkRackResolver(SparkHadoopUtil.get.conf) { override def resolve(conf: Configuration, hostName: String): String = { if (hostName == "host3") "/rack2" else "/rack1" From fa7daa48277c5ce92c93ea62906a2ebd88dfb905 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 29 Mar 2019 16:21:25 -0500 Subject: [PATCH 24/28] review feedback; remove skipRackResolution --- .../spark/scheduler/TaskSchedulerImpl.scala | 31 ++++------------- .../spark/scheduler/TaskSetManager.scala | 33 ++++++++----------- .../spark/scheduler/TaskSetManagerSuite.scala | 26 ++------------- ...yPreferredContainerPlacementStrategy.scala | 2 +- .../spark/deploy/yarn/SparkRackResolver.scala | 29 +++++++--------- .../spark/deploy/yarn/YarnAllocator.scala | 2 +- .../scheduler/cluster/YarnScheduler.scala | 6 ++-- .../deploy/yarn/YarnAllocatorSuite.scala | 4 +-- 8 files changed, 42 insertions(+), 91 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d4ffaf490f060..ee3efb6518ffd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -818,37 +818,20 @@ private[spark] class TaskSchedulerImpl( blacklistTrackerOpt.map(_.nodeBlacklist()).getOrElse(scala.collection.immutable.Set()) } - // Add a on-off switch to save time for rack resolving - private def skipRackResolving: Boolean = sc.conf.get(LOCALITY_WAIT_RACK) == 0L - - /** - * Get racks info for hosts. This is the internal method of [[getRacksForHosts]]. - * It should be override in different TaskScheduler. - * The return racks must have to be the same length as the hosts passed in. - * Return [[defaultRackValue]] sequence by default. - */ - protected def doGetRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { - hosts.map(_ => defaultRackValue) - } - // By default, rack is unknown def getRackForHost(hosts: String): Option[String] = { - if (skipRackResolving) { - defaultRackValue - } else { - doGetRacksForHosts(Seq(hosts)).head - } + getRacksForHosts(Seq(hosts)).head } /** - * null in return sequences will be replaced to [[defaultRackValue]]. + * Get racks info for hosts. + * + * The returned Sequence must have to be the same length as the hosts + * passed in. + * Return [[defaultRackValue]] sequence by default. */ def getRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { - if (skipRackResolving) { - hosts.map(_ => defaultRackValue) - } else { - doGetRacksForHosts(hosts) - } + hosts.map(_ => defaultRackValue) } private def waitBackendReady(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 87aea9fda27f9..4eb4f559eb67d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -190,19 +190,19 @@ private[spark] class TaskSetManager( private def addPendingTasks(): Unit = { val (_, duration) = Utils.timeTakenMs { - val hostToIndices = new HashMap[String, ArrayBuffer[Int]]() for (i <- (0 until numTasks).reverse) { - addPendingTask(i, initializing = true, Some(hostToIndices)) + addPendingTask(i, resolveRacks = false) } // Resolve the rack for each host. This can be slow, so de-dupe the list of hosts, // and assign the rack to all relevant task indices. - for ( - (Some(rack), indices) <- sched.getRacksForHosts(hostToIndices.keySet.toSeq) - .zip(hostToIndices.values)) { - pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) ++= indices + val racks = sched.getRacksForHosts(pendingTasksForHost.keySet.toSeq) + racks.zip(pendingTasksForHost.values).foreach { + case (Some(rack), indices) => + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) ++= indices + case (None, _) => // no rack, nothing to do } } - logDebug(s"Adding pending tasks take $duration ms") + logDebug(s"Adding pending tasks took $duration ms") } /** @@ -231,8 +231,7 @@ private[spark] class TaskSetManager( /** Add a task to all the pending-task lists that it should be on. */ private[spark] def addPendingTask( index: Int, - initializing: Boolean = false, - initializingMap: Option[HashMap[String, ArrayBuffer[Int]]] = None) { + resolveRacks: Boolean = true): Unit = { for (loc <- tasks(index).preferredLocations) { loc match { case e: ExecutorCacheTaskLocation => @@ -253,11 +252,8 @@ private[spark] class TaskSetManager( } pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer) += index - if (initializing) { - // preferredLocation -> task indices, initializingMap used when TaskSetManager initializing - initializingMap.foreach(_.getOrElseUpdate(loc.host, new ArrayBuffer) += index) - } else { - for (rack <- sched.getRackForHost(loc.host)) { + if (resolveRacks) { + sched.getRackForHost(loc.host).foreach { rack => pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index } } @@ -273,27 +269,24 @@ private[spark] class TaskSetManager( /** * Return the pending tasks list for a given executor ID, or an empty list if * there is no map entry for that host - * This is visible for testing. */ - private[scheduler] def getPendingTasksForExecutor(executorId: String): ArrayBuffer[Int] = { + private def getPendingTasksForExecutor(executorId: String): ArrayBuffer[Int] = { pendingTasksForExecutor.getOrElse(executorId, ArrayBuffer()) } /** * Return the pending tasks list for a given host, or an empty list if * there is no map entry for that host - * This is visible for testing. */ - private[scheduler] def getPendingTasksForHost(host: String): ArrayBuffer[Int] = { + private def getPendingTasksForHost(host: String): ArrayBuffer[Int] = { pendingTasksForHost.getOrElse(host, ArrayBuffer()) } /** * Return the pending rack-local task list for a given rack, or an empty list if * there is no map entry for that rack - * This is visible for testing. */ - private[scheduler] def getPendingTasksForRack(rack: String): ArrayBuffer[Int] = { + private def getPendingTasksForRack(rack: String): ArrayBuffer[Int] = { pendingTasksForRack.getOrElse(rack, ArrayBuffer()) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 2b428d498ac25..780a134d0597d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -153,7 +153,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex } } - override def doGetRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { + override def getRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { FakeRackUtil.getRacksForHosts(hosts) } } @@ -1326,7 +1326,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY) // Assert the task has been black listed on the executor it was last executed on. - when(taskSetManagerSpy.addPendingTask(anyInt(), anyBoolean(), any())).thenAnswer( + when(taskSetManagerSpy.addPendingTask(anyInt(), anyBoolean())).thenAnswer( new Answer[Unit] { override def answer(invocationOnMock: InvocationOnMock): Unit = { val task: Int = invocationOnMock.getArgument(0) @@ -1340,7 +1340,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val e = new ExceptionFailure("a", "b", Array(), "c", None) taskSetManagerSpy.handleFailedTask(taskDesc.get.taskId, TaskState.FAILED, e) - verify(taskSetManagerSpy, times(1)).addPendingTask(anyInt(), anyBoolean(), any()) + verify(taskSetManagerSpy, times(1)).addPendingTask(anyInt(), anyBoolean()) } test("SPARK-21563 context's added jars shouldn't change mid-TaskSet") { @@ -1648,24 +1648,4 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } assert(FakeRackUtil.numBatchInvocation === 1) } - - test("SPARK-13704 Rack resolving is skipped when spark.locality.wait is zero") { - val conf = new SparkConf().set(config.LOCALITY_WAIT.key, "0") - sc = new SparkContext("local", "test", conf) - for (i <- 0 to 99) { - FakeRackUtil.assignHostToRack("host" + i, "rack" + (i % 20)) - } - sched = new FakeTaskScheduler(sc, - ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) - val locations = new ArrayBuffer[Seq[TaskLocation]]() - for (i <- 0 to 99) { - locations += Seq(TaskLocation("host" + i)) - } - val taskSet = FakeTask.createTaskSet(100, locations: _*) - val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - // verify the total number not changed with SPARK-13704 - assert(manager.getPendingTasksForRack(sched.defaultRackValue.get).length === 100) - assert(FakeRackUtil.numBatchInvocation === 0) - } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala index 8cd5c1292d1ba..2288bb55d8b47 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala @@ -138,7 +138,7 @@ private[yarn] class LocalityPreferredContainerPlacementStrategy( // Only filter out the ratio which is larger than 0, which means the current host can // still be allocated with new container request. val hosts = preferredLocalityRatio.filter(_._2 > 0).keys.toArray - val racks = resolver.resolve(yarnConf, hosts).map(_.getNetworkLocation) + val racks = resolver.resolve(hosts).map(_.getNetworkLocation) .filter(_ != null).toSet containerLocalityPreferences += ContainerLocalityPreferences(hosts, racks.toArray) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index f09b6776a0a32..8ec51f02204a3 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -35,7 +35,7 @@ import org.apache.spark.internal.Logging * default behavior, since YARN's class self-initializes the first time it's called, and * future calls all use the initial configuration. */ -private[yarn] class SparkRackResolver(conf: Configuration) extends Logging { +private[spark] class SparkRackResolver(conf: Configuration) extends Logging { // RackResolver logs an INFO message whenever it resolves a rack, which is way too often. if (Logger.getLogger(classOf[RackResolver]).getLevel == null) { @@ -47,14 +47,14 @@ private[yarn] class SparkRackResolver(conf: Configuration) extends Logging { conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, classOf[ScriptBasedMapping], classOf[DNSToSwitchMapping]) ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) - .asInstanceOf[DNSToSwitchMapping] match { + .asInstanceOf[DNSToSwitchMapping] match { case c: CachedDNSToSwitchMapping => c case o => new CachedDNSToSwitchMapping(o) } } - def resolve(conf: Configuration, hostName: String): String = { - SparkRackResolver(conf).coreResolve(Seq(hostName)).head.getNetworkLocation + def resolve(hostName: String): String = { + coreResolve(Seq(hostName)).head.getNetworkLocation } /** @@ -62,8 +62,8 @@ private[yarn] class SparkRackResolver(conf: Configuration) extends Logging { * This should be changed to `RackResolver.resolve(conf, hostNames)` * in hadoop releases with YARN-9332. */ - def resolve(conf: Configuration, hostNames: Seq[String]): Seq[Node] = { - SparkRackResolver(conf).coreResolve(hostNames) + def resolve(hostNames: Seq[String]): Seq[Node] = { + coreResolve(hostNames) } private def coreResolve(hostNames: Seq[String]): Seq[Node] = { @@ -72,7 +72,7 @@ private[yarn] class SparkRackResolver(conf: Configuration) extends Logging { val rNameList = dnsToSwitchMapping.resolve(hostNames.toList.asJava).asScala if (rNameList == null || rNameList.isEmpty) { hostNames.foreach(nodes += new NodeBase(_, NetworkTopology.DEFAULT_RACK)) - logInfo(s"Got an error when resolve hostNames. " + + logInfo(s"Got an error when resolving hostNames. " + s"Falling back to ${NetworkTopology.DEFAULT_RACK} for all") } else { for ((hostName, rName) <- hostNames.zip(rNameList)) { @@ -100,15 +100,14 @@ private[yarn] class SparkRackResolver(conf: Configuration) extends Logging { * In the meantime, this is a re-implementation for spark's use. */ object SparkRackResolver extends Logging { - private var instance: SparkRackResolver = _ + @volatile private var instance: SparkRackResolver = _ /** - * It will return the static resolver instance. - * If you want to generate a separate one, please use [[get]] - * @param conf - * @return the static resolver instance + * It will return the static resolver instance. If there is already an instance, the passed + * conf is entirely ignored. If there is not a shared instance, it will create one with the + * given conf. */ - def apply(conf: Configuration): SparkRackResolver = { + def get(conf: Configuration): SparkRackResolver = { if (instance == null) { synchronized { if (instance == null) { @@ -119,8 +118,4 @@ object SparkRackResolver extends Logging { instance } - /** - * Instantiate a separate resolver with a separate config. - */ - def get(conf: Configuration): SparkRackResolver = new SparkRackResolver(conf) } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 8c6eff9915136..0b0dcd4fd68a5 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -420,7 +420,7 @@ private[yarn] class YarnAllocator( // Match remaining by rack val remainingAfterRackMatches = new ArrayBuffer[Container] for (allocatedContainer <- remainingAfterHostMatches) { - val rack = resolver.resolve(conf, allocatedContainer.getNodeId.getHost) + val rack = resolver.resolve(allocatedContainer.getNodeId.getHost) matchContainerToRequest(allocatedContainer, rack, containersToUse, remainingAfterRackMatches) } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index c67cb79975403..2129f9cd39d54 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -28,7 +28,7 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s override val defaultRackValue: Option[String] = Some(NetworkTopology.DEFAULT_RACK) - private[spark] val resolver = SparkRackResolver(sc.hadoopConfiguration) + private[spark] val resolver = new SparkRackResolver(sc.hadoopConfiguration) /** * Get racks info for a host list. @@ -36,9 +36,9 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s * @param hostPorts a list of task preferred locations to be resolved. * @return resolved rack list. */ - override def doGetRacksForHosts(hostPorts: Seq[String]): Seq[Option[String]] = { + override def getRacksForHosts(hostPorts: Seq[String]): Seq[Option[String]] = { val hosts = hostPorts.map(Utils.parseHostPort(_)._1) - resolver.resolve(sc.hadoopConfiguration, hosts).map { node => + resolver.resolve(hosts).map { node => Option(node.getNetworkLocation) } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index edf380ea7b00b..d5c120a5587cf 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -38,9 +38,9 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.SplitInfo import org.apache.spark.util.ManualClock -class MockResolver() extends SparkRackResolver(SparkHadoopUtil.get.conf) { +class MockResolver extends SparkRackResolver(SparkHadoopUtil.get.conf) { - override def resolve(conf: Configuration, hostName: String): String = { + override def resolve(hostName: String): String = { if (hostName == "host3") "/rack2" else "/rack1" } From 99ea54bdab0b8f4d48c2782831c656a6256fa9e4 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 29 Mar 2019 20:22:44 -0500 Subject: [PATCH 25/28] use shared instance --- .../main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala | 2 +- .../org/apache/spark/scheduler/cluster/YarnScheduler.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 6eff7d9072667..7c67493c33160 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -83,7 +83,7 @@ private[spark] class YarnRMClient extends Logging { localResources: Map[String, LocalResource]): YarnAllocator = { require(registered, "Must register AM before creating allocator.") new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, appAttemptId, securityMgr, - localResources, new SparkRackResolver(conf)) + localResources, SparkRackResolver.get(conf)) } /** diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index 2129f9cd39d54..8f1a1598b54e9 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -28,7 +28,7 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s override val defaultRackValue: Option[String] = Some(NetworkTopology.DEFAULT_RACK) - private[spark] val resolver = new SparkRackResolver(sc.hadoopConfiguration) + private[spark] val resolver = SparkRackResolver.get(sc.hadoopConfiguration) /** * Get racks info for a host list. From e5989842d79232b162815af4d351bd29cdd7b12e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 1 Apr 2019 14:22:48 -0500 Subject: [PATCH 26/28] review feedback --- .../spark/scheduler/TaskSchedulerImpl.scala | 4 ++-- .../spark/scheduler/TaskSetManagerSuite.scala | 22 ++++++++++++++----- 2 files changed, 18 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index e9664b0fe03ca..bf102c4cbf162 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -840,8 +840,8 @@ private[spark] class TaskSchedulerImpl( } // By default, rack is unknown - def getRackForHost(hosts: String): Option[String] = { - getRacksForHosts(Seq(hosts)).head + def getRackForHost(host: String): Option[String] = { + getRacksForHosts(Seq(host)).head } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 780a134d0597d..8182b09eafc6a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,7 +22,7 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString} +import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString, eq => meq} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -32,7 +32,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.serializer.SerializerInstance import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{AccumulatorV2, ManualClock, Utils} +import org.apache.spark.util.{AccumulatorV2, ManualClock} class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { @@ -70,10 +70,12 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) object FakeRackUtil { private val hostToRack = new mutable.HashMap[String, String]() var numBatchInvocation = 0 + var numSingleHostInvocation = 0 def cleanUp() { hostToRack.clear() numBatchInvocation = 0 + numSingleHostInvocation = 0 } def assignHostToRack(host: String, rack: String) { @@ -84,6 +86,8 @@ object FakeRackUtil { assert(hosts.toSet.size == hosts.size) // no dups in hosts if (hosts.nonEmpty && hosts.length != 1) { numBatchInvocation += 1 + } else if (hosts.length == 1) { + numSingleHostInvocation += 1 } hosts.map(hostToRack.get(_)) } @@ -1340,7 +1344,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val e = new ExceptionFailure("a", "b", Array(), "c", None) taskSetManagerSpy.handleFailedTask(taskDesc.get.taskId, TaskState.FAILED, e) - verify(taskSetManagerSpy, times(1)).addPendingTask(anyInt(), anyBoolean()) + verify(taskSetManagerSpy, times(1)).addPendingTask(meq(0), meq(false)) } test("SPARK-21563 context's added jars shouldn't change mid-TaskSet") { @@ -1615,8 +1619,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("SPARK-13704 Rack Resolution is done with a batch of de-duped hosts") { val conf = new SparkConf() - .set(config.LOCALITY_WAIT.key, "0") - .set(config.LOCALITY_WAIT_RACK.key, "1s") + .set(config.LOCALITY_WAIT, 0L) + .set(config.LOCALITY_WAIT_RACK, 1L) sc = new SparkContext("local", "test", conf) // Create a cluster with 20 racks, with hosts spread out among them val execAndHost = (0 to 199).map { i => @@ -1631,7 +1635,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } val taskSet = FakeTask.createTaskSet(100, locations: _*) val clock = new ManualClock + // make sure we only do one rack resolution call, for the entire batch of hosts, as this + // can be expensive. the FakeTaskScheduler calls rack resolution more than the real one + // -- that is outside of the scope of this test, we just want to check the task set manager. + FakeRackUtil.numBatchInvocation = 0 + FakeRackUtil.numSingleHostInvocation = 0 val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + assert(FakeRackUtil.numBatchInvocation === 1) + assert(FakeRackUtil.numSingleHostInvocation === 0) // with rack locality, reject an offer on a host with an unknown rack assert(manager.resourceOffer("otherExec", "otherHost", TaskLocality.RACK_LOCAL).isEmpty) (0 until 20).foreach { rackIdx => @@ -1646,6 +1657,5 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg .isDefined) } } - assert(FakeRackUtil.numBatchInvocation === 1) } } From f5efc7418248dd4f9b2b3f4447963d4e2686ae77 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 1 Apr 2019 14:37:25 -0500 Subject: [PATCH 27/28] fix --- .../org/apache/spark/scheduler/TaskSetManagerSuite.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 8182b09eafc6a..e714a991a3e4b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -1636,7 +1636,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(100, locations: _*) val clock = new ManualClock // make sure we only do one rack resolution call, for the entire batch of hosts, as this - // can be expensive. the FakeTaskScheduler calls rack resolution more than the real one + // can be expensive. The FakeTaskScheduler calls rack resolution more than the real one // -- that is outside of the scope of this test, we just want to check the task set manager. FakeRackUtil.numBatchInvocation = 0 FakeRackUtil.numSingleHostInvocation = 0 @@ -1657,5 +1657,9 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg .isDefined) } } + // check no more expensive calls to the rack resolution. manager.resourceOffer() will call + // the single-host resolution, but the real rack resolution would have cached all hosts + // by that point. + assert(FakeRackUtil.numBatchInvocation === 1) } } From ad63e154b0aae43e2e83fc2ea7700b0545c2fd99 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 3 Apr 2019 14:25:29 -0500 Subject: [PATCH 28/28] review feedback --- .../spark/scheduler/TaskSchedulerImpl.scala | 18 +++++++++--------- .../spark/scheduler/TaskSetManager.scala | 5 +++-- .../spark/scheduler/TaskSetManagerSuite.scala | 4 ++-- .../spark/deploy/yarn/SparkRackResolver.scala | 11 +++-------- .../scheduler/cluster/YarnScheduler.scala | 6 ------ 5 files changed, 17 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index cc24112c01922..e401c395a0486 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -158,10 +158,6 @@ private[spark] class TaskSchedulerImpl( private[scheduler] var barrierCoordinator: RpcEndpoint = null - /** - * It can be override in different TaskScheduler, like Yarn. - * None by default. This should be initialized before any invocation. - */ protected val defaultRackValue: Option[String] = None private def maybeInitBarrierCoordinator(): Unit = { @@ -837,17 +833,21 @@ private[spark] class TaskSchedulerImpl( blacklistTrackerOpt.map(_.nodeBlacklist()).getOrElse(Set.empty) } - // By default, rack is unknown + /** + * Get the rack for one host. + * + * Note that [[getRacksForHosts]] should be preferred when possible as that can be much + * more efficient. + */ def getRackForHost(host: String): Option[String] = { getRacksForHosts(Seq(host)).head } /** - * Get racks info for hosts. + * Get racks for multiple hosts. * - * The returned Sequence must have to be the same length as the hosts - * passed in. - * Return [[defaultRackValue]] sequence by default. + * The returned Sequence will be the same length as the hosts argument and can be zipped + * together with the hosts argument. */ def getRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { hosts.map(_ => defaultRackValue) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index cb882a3cc63e3..144422022c22f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -195,8 +195,9 @@ private[spark] class TaskSetManager( } // Resolve the rack for each host. This can be slow, so de-dupe the list of hosts, // and assign the rack to all relevant task indices. - val racks = sched.getRacksForHosts(pendingTasksForHost.keySet.toSeq) - racks.zip(pendingTasksForHost.values).foreach { + val (hosts, indicesForHosts) = pendingTasksForHost.toSeq.unzip + val racks = sched.getRacksForHosts(hosts) + racks.zip(indicesForHosts).foreach { case (Some(rack), indices) => pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) ++= indices case (None, _) => // no rack, nothing to do diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 318b6f1ac22dd..79160d05b3e60 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,7 +22,7 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString, eq => meq} +import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock @@ -1337,7 +1337,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val e = new ExceptionFailure("a", "b", Array(), "c", None) taskSetManagerSpy.handleFailedTask(taskDesc.get.taskId, TaskState.FAILED, e) - verify(taskSetManagerSpy, times(1)).addPendingTask(meq(0), meq(false)) + verify(taskSetManagerSpy, times(1)).addPendingTask(0, false) } test("SPARK-21563 context's added jars shouldn't change mid-TaskSet") { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index 8ec51f02204a3..cab32724e13a6 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -31,9 +31,9 @@ import org.apache.log4j.{Level, Logger} import org.apache.spark.internal.Logging /** - * Re-implement YARN's [[RackResolver]]. This allows Spark tests to easily override the - * default behavior, since YARN's class self-initializes the first time it's called, and - * future calls all use the initial configuration. + * Re-implement YARN's [[RackResolver]] for hadoop releases without YARN-9332. + * This also allows Spark tests to easily override the default behavior, since YARN's class + * self-initializes the first time it's called, and future calls all use the initial configuration. */ private[spark] class SparkRackResolver(conf: Configuration) extends Logging { @@ -93,11 +93,6 @@ private[spark] class SparkRackResolver(conf: Configuration) extends Logging { * Utility to resolve the rack for hosts in an efficient manner. * It will cache the rack for individual hosts to avoid * repeatedly performing the same expensive lookup. - * - * Its logic refers [[org.apache.hadoop.yarn.util.RackResolver]] and enhanced. - * This will be unnecessary in hadoop releases with YARN-9332. - * With that, we could just directly use [[org.apache.hadoop.yarn.util.RackResolver]]. - * In the meantime, this is a re-implementation for spark's use. */ object SparkRackResolver extends Logging { @volatile private var instance: SparkRackResolver = _ diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index 8f1a1598b54e9..d466ed77a929e 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -30,12 +30,6 @@ private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(s private[spark] val resolver = SparkRackResolver.get(sc.hadoopConfiguration) - /** - * Get racks info for a host list. - * Use [[SparkRackResolver]] to resolve racks before the dependency Hadoop applied YARN-9332 - * @param hostPorts a list of task preferred locations to be resolved. - * @return resolved rack list. - */ override def getRacksForHosts(hostPorts: Seq[String]): Seq[Option[String]] = { val hosts = hostPorts.map(Utils.parseHostPort(_)._1) resolver.resolve(hosts).map { node =>