Repository: spark
Updated Branches:
  refs/heads/branch-2.1 f9546dacb -> 00c124884


[SPARK-20191][YARN] Crate wrapper for RackResolver so tests can override it.

Current test code tries to override the RackResolver used by setting
configuration params, but because YARN libs statically initialize the
resolver the first time it's used, that means that those configs don't
really take effect during Spark tests.

This change adds a wrapper class that easily allows tests to override the
behavior of the resolver for the Spark code that uses it.

Author: Marcelo Vanzin <van...@cloudera.com>

Closes #17508 from vanzin/SPARK-20191.

(cherry picked from commit 0736980f395f114faccbd58e78280ca63ed289c7)
Signed-off-by: Marcelo Vanzin <van...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/00c12488
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/00c12488
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/00c12488

Branch: refs/heads/branch-2.1
Commit: 00c124884d84f15cbd930136ceb21c912305c815
Parents: f9546da
Author: Marcelo Vanzin <van...@cloudera.com>
Authored: Tue Apr 4 11:38:05 2017 -0700
Committer: Marcelo Vanzin <van...@cloudera.com>
Committed: Tue Apr 4 11:47:43 2017 -0700

----------------------------------------------------------------------
 ...ityPreferredContainerPlacementStrategy.scala |  6 +--
 .../spark/deploy/yarn/SparkRackResolver.scala   | 40 ++++++++++++++++++++
 .../spark/deploy/yarn/YarnAllocator.scala       | 13 ++-----
 .../apache/spark/deploy/yarn/YarnRMClient.scala |  2 +-
 .../yarn/LocalityPlacementStrategySuite.scala   |  8 +---
 .../spark/deploy/yarn/YarnAllocatorSuite.scala  | 22 +++--------
 6 files changed, 56 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/00c12488/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
----------------------------------------------------------------------
diff --git 
a/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
 
b/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
index db638d8..872fd35 100644
--- 
a/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
+++ 
b/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
@@ -23,7 +23,6 @@ import scala.collection.JavaConverters._
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.yarn.api.records.{ContainerId, Resource}
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
-import org.apache.hadoop.yarn.util.RackResolver
 
 import org.apache.spark.SparkConf
 import org.apache.spark.internal.config._
@@ -83,7 +82,8 @@ private[yarn] case class ContainerLocalityPreferences(nodes: 
Array[String], rack
 private[yarn] class LocalityPreferredContainerPlacementStrategy(
     val sparkConf: SparkConf,
     val yarnConf: Configuration,
-    val resource: Resource) {
+    val resource: Resource,
+    resolver: SparkRackResolver) {
 
   /**
    * Calculate each container's node locality and rack locality
@@ -139,7 +139,7 @@ private[yarn] class 
LocalityPreferredContainerPlacementStrategy(
         // still be allocated with new container request.
         val hosts = preferredLocalityRatio.filter(_._2 > 0).keys.toArray
         val racks = hosts.map { h =>
-          RackResolver.resolve(yarnConf, h).getNetworkLocation
+          resolver.resolve(yarnConf, h)
         }.toSet
         containerLocalityPreferences += ContainerLocalityPreferences(hosts, 
racks.toArray)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/00c12488/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala
----------------------------------------------------------------------
diff --git 
a/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala 
b/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala
new file mode 100644
index 0000000..c711d08
--- /dev/null
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.yarn
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.util.RackResolver
+import org.apache.log4j.{Level, Logger}
+
+/**
+ * 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
+ * future calls all use the initial configuration.
+ */
+private[yarn] class SparkRackResolver {
+
+  // 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)
+  }
+
+  def resolve(conf: Configuration, hostName: String): String = {
+    RackResolver.resolve(conf, hostName).getNetworkLocation()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/00c12488/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
----------------------------------------------------------------------
diff --git 
a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala 
b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
index 0b66d1c..639e564 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.api.records._
 import org.apache.hadoop.yarn.client.api.AMRMClient
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
 import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.yarn.util.RackResolver
 import org.apache.log4j.{Level, Logger}
 
 import org.apache.spark.{SecurityManager, SparkConf, SparkException}
@@ -65,16 +64,12 @@ private[yarn] class YarnAllocator(
     amClient: AMRMClient[ContainerRequest],
     appAttemptId: ApplicationAttemptId,
     securityMgr: SecurityManager,
-    localResources: Map[String, LocalResource])
+    localResources: Map[String, LocalResource],
+    resolver: SparkRackResolver)
   extends Logging {
 
   import YarnAllocator._
 
-  // 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)
-  }
-
   // Visible for testing.
   val allocatedHostToContainersMap = new HashMap[String, 
collection.mutable.Set[ContainerId]]
   val allocatedContainerToHostMap = new HashMap[ContainerId, String]
@@ -171,7 +166,7 @@ private[yarn] class YarnAllocator(
 
   // A container placement strategy based on pending tasks' locality preference
   private[yarn] val containerPlacementStrategy =
-    new LocalityPreferredContainerPlacementStrategy(sparkConf, conf, resource)
+    new LocalityPreferredContainerPlacementStrategy(sparkConf, conf, resource, 
resolver)
 
   /**
    * Use a different clock for YarnAllocator. This is mainly used for testing.
@@ -422,7 +417,7 @@ private[yarn] class YarnAllocator(
     // Match remaining by rack
     val remainingAfterRackMatches = new ArrayBuffer[Container]
     for (allocatedContainer <- remainingAfterHostMatches) {
-      val rack = RackResolver.resolve(conf, 
allocatedContainer.getNodeId.getHost).getNetworkLocation
+      val rack = resolver.resolve(conf, allocatedContainer.getNodeId.getHost)
       matchContainerToRequest(allocatedContainer, rack, containersToUse,
         remainingAfterRackMatches)
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/00c12488/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
----------------------------------------------------------------------
diff --git 
a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala 
b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
index 53df11e..9e14d63 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
@@ -75,7 +75,7 @@ private[spark] class YarnRMClient extends Logging {
       registered = true
     }
     new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, 
getAttemptId(), securityMgr,
-      localResources)
+      localResources, new SparkRackResolver())
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/00c12488/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala
----------------------------------------------------------------------
diff --git 
a/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala
 
b/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala
index fb80ff9..b7f2565 100644
--- 
a/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala
+++ 
b/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala
@@ -17,10 +17,9 @@
 
 package org.apache.spark.deploy.yarn
 
+import scala.collection.JavaConverters._
 import scala.collection.mutable.{HashMap, HashSet, Set}
 
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic
-import org.apache.hadoop.net.DNSToSwitchMapping
 import org.apache.hadoop.yarn.api.records._
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.mockito.Mockito._
@@ -51,9 +50,6 @@ class LocalityPlacementStrategySuite extends SparkFunSuite {
 
   private def runTest(): Unit = {
     val yarnConf = new YarnConfiguration()
-    yarnConf.setClass(
-      CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
-      classOf[MockResolver], classOf[DNSToSwitchMapping])
 
     // The numbers below have been chosen to balance being large enough to 
replicate the
     // original issue while not taking too long to run when the issue is 
fixed. The main
@@ -62,7 +58,7 @@ class LocalityPlacementStrategySuite extends SparkFunSuite {
 
     val resource = Resource.newInstance(8 * 1024, 4)
     val strategy = new LocalityPreferredContainerPlacementStrategy(new 
SparkConf(),
-      yarnConf, resource)
+      yarnConf, resource, new MockResolver())
 
     val totalTasks = 32 * 1024
     val totalContainers = totalTasks / 16

http://git-wip-us.apache.org/repos/asf/spark/blob/00c12488/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala
----------------------------------------------------------------------
diff --git 
a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala 
b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala
index 994dc75..1b3f438 100644
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala
+++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala
@@ -17,10 +17,7 @@
 
 package org.apache.spark.deploy.yarn
 
-import java.util.{Arrays, List => JList}
-
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic
-import org.apache.hadoop.net.DNSToSwitchMapping
+import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.yarn.api.records._
 import org.apache.hadoop.yarn.client.api.AMRMClient
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
@@ -36,24 +33,16 @@ import org.apache.spark.rpc.RpcEndpointRef
 import org.apache.spark.scheduler.SplitInfo
 import org.apache.spark.util.ManualClock
 
-class MockResolver extends DNSToSwitchMapping {
+class MockResolver extends SparkRackResolver {
 
-  override def resolve(names: JList[String]): JList[String] = {
-    if (names.size > 0 && names.get(0) == "host3") Arrays.asList("/rack2")
-    else Arrays.asList("/rack1")
+  override def resolve(conf: Configuration, hostName: String): String = {
+    if (hostName == "host3") "/rack2" else "/rack1"
   }
 
-  override def reloadCachedMappings() {}
-
-  def reloadCachedMappings(names: JList[String]) {}
 }
 
 class YarnAllocatorSuite extends SparkFunSuite with Matchers with 
BeforeAndAfterEach {
   val conf = new YarnConfiguration()
-  conf.setClass(
-    CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
-    classOf[MockResolver], classOf[DNSToSwitchMapping])
-
   val sparkConf = new SparkConf()
   sparkConf.set("spark.driver.host", "localhost")
   sparkConf.set("spark.driver.port", "4040")
@@ -107,7 +96,8 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers 
with BeforeAndAfter
       rmClient,
       appAttemptId,
       new SecurityManager(sparkConf),
-      Map())
+      Map(),
+      new MockResolver())
   }
 
   def createContainer(host: String): Container = {


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

Reply via email to