You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by va...@apache.org on 2016/12/07 00:23:38 UTC

[01/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Repository: spark
Updated Branches:
  refs/heads/master a8ced76f1 -> 81e5619ca


http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala
deleted file mode 100644
index a58784f..0000000
--- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala
+++ /dev/null
@@ -1,372 +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.network.yarn
-
-import java.io.{DataOutputStream, File, FileOutputStream, IOException}
-import java.nio.ByteBuffer
-import java.nio.file.Files
-import java.nio.file.attribute.PosixFilePermission._
-import java.util.EnumSet
-
-import scala.annotation.tailrec
-import scala.concurrent.duration._
-import scala.language.postfixOps
-
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.service.ServiceStateException
-import org.apache.hadoop.yarn.api.records.ApplicationId
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.yarn.server.api.{ApplicationInitializationContext, ApplicationTerminationContext}
-import org.scalatest.{BeforeAndAfterEach, Matchers}
-import org.scalatest.concurrent.Eventually._
-
-import org.apache.spark.SecurityManager
-import org.apache.spark.SparkFunSuite
-import org.apache.spark.network.shuffle.ShuffleTestAccessor
-import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo
-import org.apache.spark.util.Utils
-
-class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach {
-  private[yarn] var yarnConfig: YarnConfiguration = null
-  private[yarn] val SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"
-
-  override def beforeEach(): Unit = {
-    super.beforeEach()
-    yarnConfig = new YarnConfiguration()
-    yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle")
-    yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"),
-      classOf[YarnShuffleService].getCanonicalName)
-    yarnConfig.setInt("spark.shuffle.service.port", 0)
-    yarnConfig.setBoolean(YarnShuffleService.STOP_ON_FAILURE_KEY, true)
-    val localDir = Utils.createTempDir()
-    yarnConfig.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath)
-  }
-
-  var s1: YarnShuffleService = null
-  var s2: YarnShuffleService = null
-  var s3: YarnShuffleService = null
-
-  override def afterEach(): Unit = {
-    try {
-      if (s1 != null) {
-        s1.stop()
-        s1 = null
-      }
-      if (s2 != null) {
-        s2.stop()
-        s2 = null
-      }
-      if (s3 != null) {
-        s3.stop()
-        s3 = null
-      }
-    } finally {
-      super.afterEach()
-    }
-  }
-
-  test("executor state kept across NM restart") {
-    s1 = new YarnShuffleService
-    // set auth to true to test the secrets recovery
-    yarnConfig.setBoolean(SecurityManager.SPARK_AUTH_CONF, true)
-    s1.init(yarnConfig)
-    val app1Id = ApplicationId.newInstance(0, 1)
-    val app1Data = makeAppInfo("user", app1Id)
-    s1.initializeApplication(app1Data)
-    val app2Id = ApplicationId.newInstance(0, 2)
-    val app2Data = makeAppInfo("user", app2Id)
-    s1.initializeApplication(app2Data)
-
-    val execStateFile = s1.registeredExecutorFile
-    execStateFile should not be (null)
-    val secretsFile = s1.secretsFile
-    secretsFile should not be (null)
-    val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER)
-    val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER)
-
-    val blockHandler = s1.blockHandler
-    val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler)
-    ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile)
-
-    blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1)
-    blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2)
-    ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", blockResolver) should
-      be (Some(shuffleInfo1))
-    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", blockResolver) should
-      be (Some(shuffleInfo2))
-
-    if (!execStateFile.exists()) {
-      @tailrec def findExistingParent(file: File): File = {
-        if (file == null) file
-        else if (file.exists()) file
-        else findExistingParent(file.getParentFile())
-      }
-      val existingParent = findExistingParent(execStateFile)
-      assert(false, s"$execStateFile does not exist -- closest existing parent is $existingParent")
-    }
-    assert(execStateFile.exists(), s"$execStateFile did not exist")
-
-    // now we pretend the shuffle service goes down, and comes back up
-    s1.stop()
-    s2 = new YarnShuffleService
-    s2.init(yarnConfig)
-    s2.secretsFile should be (secretsFile)
-    s2.registeredExecutorFile should be (execStateFile)
-
-    val handler2 = s2.blockHandler
-    val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2)
-
-    // now we reinitialize only one of the apps, and expect yarn to tell us that app2 was stopped
-    // during the restart
-    s2.initializeApplication(app1Data)
-    s2.stopApplication(new ApplicationTerminationContext(app2Id))
-    ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (Some(shuffleInfo1))
-    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None)
-
-    // Act like the NM restarts one more time
-    s2.stop()
-    s3 = new YarnShuffleService
-    s3.init(yarnConfig)
-    s3.registeredExecutorFile should be (execStateFile)
-    s3.secretsFile should be (secretsFile)
-
-    val handler3 = s3.blockHandler
-    val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3)
-
-    // app1 is still running
-    s3.initializeApplication(app1Data)
-    ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver3) should be (Some(shuffleInfo1))
-    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (None)
-    s3.stop()
-  }
-
-  test("removed applications should not be in registered executor file") {
-    s1 = new YarnShuffleService
-    yarnConfig.setBoolean(SecurityManager.SPARK_AUTH_CONF, false)
-    s1.init(yarnConfig)
-    val secretsFile = s1.secretsFile
-    secretsFile should be (null)
-    val app1Id = ApplicationId.newInstance(0, 1)
-    val app1Data = makeAppInfo("user", app1Id)
-    s1.initializeApplication(app1Data)
-    val app2Id = ApplicationId.newInstance(0, 2)
-    val app2Data = makeAppInfo("user", app2Id)
-    s1.initializeApplication(app2Data)
-
-    val execStateFile = s1.registeredExecutorFile
-    execStateFile should not be (null)
-    val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER)
-    val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER)
-
-    val blockHandler = s1.blockHandler
-    val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler)
-    ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile)
-
-    blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1)
-    blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2)
-
-    val db = ShuffleTestAccessor.shuffleServiceLevelDB(blockResolver)
-    ShuffleTestAccessor.reloadRegisteredExecutors(db) should not be empty
-
-    s1.stopApplication(new ApplicationTerminationContext(app1Id))
-    ShuffleTestAccessor.reloadRegisteredExecutors(db) should not be empty
-    s1.stopApplication(new ApplicationTerminationContext(app2Id))
-    ShuffleTestAccessor.reloadRegisteredExecutors(db) shouldBe empty
-  }
-
-  test("shuffle service should be robust to corrupt registered executor file") {
-    s1 = new YarnShuffleService
-    s1.init(yarnConfig)
-    val app1Id = ApplicationId.newInstance(0, 1)
-    val app1Data = makeAppInfo("user", app1Id)
-    s1.initializeApplication(app1Data)
-
-    val execStateFile = s1.registeredExecutorFile
-    val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER)
-
-    val blockHandler = s1.blockHandler
-    val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler)
-    ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile)
-
-    blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1)
-
-    // now we pretend the shuffle service goes down, and comes back up.  But we'll also
-    // make a corrupt registeredExecutor File
-    s1.stop()
-
-    execStateFile.listFiles().foreach{_.delete()}
-
-    val out = new DataOutputStream(new FileOutputStream(execStateFile + "/CURRENT"))
-    out.writeInt(42)
-    out.close()
-
-    s2 = new YarnShuffleService
-    s2.init(yarnConfig)
-    s2.registeredExecutorFile should be (execStateFile)
-
-    val handler2 = s2.blockHandler
-    val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2)
-
-    // we re-initialize app1, but since the file was corrupt there is nothing we can do about it ...
-    s2.initializeApplication(app1Data)
-    // however, when we initialize a totally new app2, everything is still happy
-    val app2Id = ApplicationId.newInstance(0, 2)
-    val app2Data = makeAppInfo("user", app2Id)
-    s2.initializeApplication(app2Data)
-    val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER)
-    resolver2.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2)
-    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (Some(shuffleInfo2))
-    s2.stop()
-
-    // another stop & restart should be fine though (eg., we recover from previous corruption)
-    s3 = new YarnShuffleService
-    s3.init(yarnConfig)
-    s3.registeredExecutorFile should be (execStateFile)
-    val handler3 = s3.blockHandler
-    val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3)
-
-    s3.initializeApplication(app2Data)
-    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (Some(shuffleInfo2))
-    s3.stop()
-  }
-
-  test("get correct recovery path") {
-    // Test recovery path is set outside the shuffle service, this is to simulate NM recovery
-    // enabled scenario, where recovery path will be set by yarn.
-    s1 = new YarnShuffleService
-    val recoveryPath = new Path(Utils.createTempDir().toURI)
-    s1.setRecoveryPath(recoveryPath)
-
-    s1.init(yarnConfig)
-    s1._recoveryPath should be (recoveryPath)
-    s1.stop()
-
-    // Test recovery path is set inside the shuffle service, this will be happened when NM
-    // recovery is not enabled or there's no NM recovery (Hadoop 2.5-).
-    s2 = new YarnShuffleService
-    s2.init(yarnConfig)
-    s2._recoveryPath should be
-      (new Path(yarnConfig.getTrimmedStrings("yarn.nodemanager.local-dirs")(0)))
-    s2.stop()
-  }
-
-  test("moving recovery file from NM local dir to recovery path") {
-    // This is to test when Hadoop is upgrade to 2.5+ and NM recovery is enabled, we should move
-    // old recovery file to the new path to keep compatibility
-
-    // Simulate s1 is running on old version of Hadoop in which recovery file is in the NM local
-    // dir.
-    s1 = new YarnShuffleService
-    // set auth to true to test the secrets recovery
-    yarnConfig.setBoolean(SecurityManager.SPARK_AUTH_CONF, true)
-    s1.init(yarnConfig)
-    val app1Id = ApplicationId.newInstance(0, 1)
-    val app1Data = makeAppInfo("user", app1Id)
-    s1.initializeApplication(app1Data)
-    val app2Id = ApplicationId.newInstance(0, 2)
-    val app2Data = makeAppInfo("user", app2Id)
-    s1.initializeApplication(app2Data)
-
-    assert(s1.secretManager.getSecretKey(app1Id.toString()) != null)
-    assert(s1.secretManager.getSecretKey(app2Id.toString()) != null)
-
-    val execStateFile = s1.registeredExecutorFile
-    execStateFile should not be (null)
-    val secretsFile = s1.secretsFile
-    secretsFile should not be (null)
-    val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER)
-    val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER)
-
-    val blockHandler = s1.blockHandler
-    val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler)
-    ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile)
-
-    blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1)
-    blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2)
-    ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", blockResolver) should
-      be (Some(shuffleInfo1))
-    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", blockResolver) should
-      be (Some(shuffleInfo2))
-
-    assert(execStateFile.exists(), s"$execStateFile did not exist")
-
-    s1.stop()
-
-    // Simulate s2 is running on Hadoop 2.5+ with NM recovery is enabled.
-    assert(execStateFile.exists())
-    val recoveryPath = new Path(Utils.createTempDir().toURI)
-    s2 = new YarnShuffleService
-    s2.setRecoveryPath(recoveryPath)
-    s2.init(yarnConfig)
-
-    // Ensure that s2 has loaded known apps from the secrets db.
-    assert(s2.secretManager.getSecretKey(app1Id.toString()) != null)
-    assert(s2.secretManager.getSecretKey(app2Id.toString()) != null)
-
-    val execStateFile2 = s2.registeredExecutorFile
-    val secretsFile2 = s2.secretsFile
-
-    recoveryPath.toString should be (new Path(execStateFile2.getParentFile.toURI).toString)
-    recoveryPath.toString should be (new Path(secretsFile2.getParentFile.toURI).toString)
-    eventually(timeout(10 seconds), interval(5 millis)) {
-      assert(!execStateFile.exists())
-    }
-    eventually(timeout(10 seconds), interval(5 millis)) {
-      assert(!secretsFile.exists())
-    }
-
-    val handler2 = s2.blockHandler
-    val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2)
-
-    // now we reinitialize only one of the apps, and expect yarn to tell us that app2 was stopped
-    // during the restart
-    // Since recovery file is got from old path, so the previous state should be stored.
-    s2.initializeApplication(app1Data)
-    s2.stopApplication(new ApplicationTerminationContext(app2Id))
-    ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (Some(shuffleInfo1))
-    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None)
-
-    s2.stop()
-  }
-
-  test("service throws error if cannot start") {
-    // Set up a read-only local dir.
-    val roDir = Utils.createTempDir()
-    Files.setPosixFilePermissions(roDir.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE))
-    yarnConfig.set(YarnConfiguration.NM_LOCAL_DIRS, roDir.getAbsolutePath())
-
-    // Try to start the shuffle service, it should fail.
-    val service = new YarnShuffleService()
-
-    try {
-      val error = intercept[ServiceStateException] {
-        service.init(yarnConfig)
-      }
-      assert(error.getCause().isInstanceOf[IOException])
-    } finally {
-      service.stop()
-      Files.setPosixFilePermissions(roDir.toPath(),
-        EnumSet.of(OWNER_READ, OWNER_WRITE, OWNER_EXECUTE))
-    }
-  }
-
-  private def makeAppInfo(user: String, appId: ApplicationId): ApplicationInitializationContext = {
-    val secret = ByteBuffer.wrap(new Array[Byte](0))
-    new ApplicationInitializationContext(user, appId, secret)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala
deleted file mode 100644
index db322cd..0000000
--- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala
+++ /dev/null
@@ -1,37 +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.network.yarn
-
-import java.io.File
-
-/**
- * just a cheat to get package-visible members in tests
- */
-object YarnTestAccessor {
-  def getShuffleServicePort: Int = {
-    YarnShuffleService.boundPort
-  }
-
-  def getShuffleServiceInstance: YarnShuffleService = {
-    YarnShuffleService.instance
-  }
-
-  def getRegisteredExecutorFile(service: YarnShuffleService): File = {
-    service.registeredExecutorFile
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala
deleted file mode 100644
index 6ea7984..0000000
--- a/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala
+++ /dev/null
@@ -1,72 +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 org.scalatest.BeforeAndAfter
-
-import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite}
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.internal.Logging
-
-/**
- * Test the integration with [[SchedulerExtensionServices]]
- */
-class ExtensionServiceIntegrationSuite extends SparkFunSuite
-  with LocalSparkContext with BeforeAndAfter
-  with Logging {
-
-  val applicationId = new StubApplicationId(0, 1111L)
-  val attemptId = new StubApplicationAttemptId(applicationId, 1)
-
-  /*
-   * Setup phase creates the spark context
-   */
-  before {
-    val sparkConf = new SparkConf()
-    sparkConf.set(SCHEDULER_SERVICES, Seq(classOf[SimpleExtensionService].getName()))
-    sparkConf.setMaster("local").setAppName("ExtensionServiceIntegrationSuite")
-    sc = new SparkContext(sparkConf)
-  }
-
-  test("Instantiate") {
-    val services = new SchedulerExtensionServices()
-    assertResult(Nil, "non-nil service list") {
-      services.getServices
-    }
-    services.start(SchedulerExtensionServiceBinding(sc, applicationId))
-    services.stop()
-  }
-
-  test("Contains SimpleExtensionService Service") {
-    val services = new SchedulerExtensionServices()
-    try {
-      services.start(SchedulerExtensionServiceBinding(sc, applicationId))
-      val serviceList = services.getServices
-      assert(serviceList.nonEmpty, "empty service list")
-      val (service :: Nil) = serviceList
-      val simpleService = service.asInstanceOf[SimpleExtensionService]
-      assert(simpleService.started.get, "service not started")
-      services.stop()
-      assert(!simpleService.started.get, "service not stopped")
-    } finally {
-      services.stop()
-    }
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala b/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala
deleted file mode 100644
index 9b8c98c..0000000
--- a/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala
+++ /dev/null
@@ -1,34 +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 java.util.concurrent.atomic.AtomicBoolean
-
-private[spark] class SimpleExtensionService extends SchedulerExtensionService {
-
-  /** started flag; set in the `start()` call, stopped in `stop()`. */
-  val started = new AtomicBoolean(false)
-
-  override def start(binding: SchedulerExtensionServiceBinding): Unit = {
-    started.set(true)
-  }
-
-  override def stop(): Unit = {
-    started.set(false)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala b/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala
deleted file mode 100644
index 4b57b95..0000000
--- a/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala
+++ /dev/null
@@ -1,48 +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 org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
-
-/**
- * A stub application ID; can be set in constructor and/or updated later.
- * @param applicationId application ID
- * @param attempt an attempt counter
- */
-class StubApplicationAttemptId(var applicationId: ApplicationId, var attempt: Int)
-    extends ApplicationAttemptId {
-
-  override def setApplicationId(appID: ApplicationId): Unit = {
-    applicationId = appID
-  }
-
-  override def getAttemptId: Int = {
-    attempt
-  }
-
-  override def setAttemptId(attemptId: Int): Unit = {
-    attempt = attemptId
-  }
-
-  override def getApplicationId: ApplicationId = {
-    applicationId
-  }
-
-  override def build(): Unit = {
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala b/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala
deleted file mode 100644
index bffa0e0..0000000
--- a/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala
+++ /dev/null
@@ -1,42 +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 org.apache.hadoop.yarn.api.records.ApplicationId
-
-/**
- * Simple Testing Application Id; ID and cluster timestamp are set in constructor
- * and cannot be updated.
- * @param id app id
- * @param clusterTimestamp timestamp
- */
-private[spark] class StubApplicationId(id: Int, clusterTimestamp: Long) extends ApplicationId {
-  override def getId: Int = {
-    id
-  }
-
-  override def getClusterTimestamp: Long = {
-    clusterTimestamp
-  }
-
-  override def setId(id: Int): Unit = {}
-
-  override def setClusterTimestamp(clusterTimestamp: Long): Unit = {}
-
-  override def build(): Unit = {}
-}


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


[12/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala
new file mode 100644
index 0000000..ec47ab1
--- /dev/null
+++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala
@@ -0,0 +1,256 @@
+/*
+ * 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.mesos
+
+import scala.collection.JavaConverters._
+import scala.language.reflectiveCalls
+
+import org.apache.mesos.Protos.{Resource, Value}
+import org.mockito.Mockito._
+import org.scalatest._
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
+import org.apache.spark.internal.config._
+
+class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoSugar {
+
+  // scalastyle:off structural.type
+  // this is the documented way of generating fixtures in scalatest
+  def fixture: Object {val sc: SparkContext; val sparkConf: SparkConf} = new {
+    val sparkConf = new SparkConf
+    val sc = mock[SparkContext]
+    when(sc.conf).thenReturn(sparkConf)
+  }
+
+  private def createTestPortResource(range: (Long, Long), role: Option[String] = None): Resource = {
+    val rangeValue = Value.Range.newBuilder()
+    rangeValue.setBegin(range._1)
+    rangeValue.setEnd(range._2)
+    val builder = Resource.newBuilder()
+      .setName("ports")
+      .setType(Value.Type.RANGES)
+      .setRanges(Value.Ranges.newBuilder().addRange(rangeValue))
+
+    role.foreach { r => builder.setRole(r) }
+    builder.build()
+  }
+
+  private def rangesResourcesToTuple(resources: List[Resource]): List[(Long, Long)] = {
+    resources.flatMap{resource => resource.getRanges.getRangeList
+      .asScala.map(range => (range.getBegin, range.getEnd))}
+  }
+
+  def arePortsEqual(array1: Array[(Long, Long)], array2: Array[(Long, Long)])
+    : Boolean = {
+    array1.sortBy(identity).deep == array2.sortBy(identity).deep
+  }
+
+  def arePortsEqual(array1: Array[Long], array2: Array[Long])
+    : Boolean = {
+    array1.sortBy(identity).deep == array2.sortBy(identity).deep
+  }
+
+  def getRangesFromResources(resources: List[Resource]): List[(Long, Long)] = {
+    resources.flatMap{ resource =>
+      resource.getRanges.getRangeList.asScala.toList.map{
+        range => (range.getBegin, range.getEnd)}}
+  }
+
+  val utils = new MesosSchedulerUtils { }
+  // scalastyle:on structural.type
+
+  test("use at-least minimum overhead") {
+    val f = fixture
+    when(f.sc.executorMemory).thenReturn(512)
+    utils.executorMemory(f.sc) shouldBe 896
+  }
+
+  test("use overhead if it is greater than minimum value") {
+    val f = fixture
+    when(f.sc.executorMemory).thenReturn(4096)
+    utils.executorMemory(f.sc) shouldBe 4505
+  }
+
+  test("use spark.mesos.executor.memoryOverhead (if set)") {
+    val f = fixture
+    when(f.sc.executorMemory).thenReturn(1024)
+    f.sparkConf.set("spark.mesos.executor.memoryOverhead", "512")
+    utils.executorMemory(f.sc) shouldBe 1536
+  }
+
+  test("parse a non-empty constraint string correctly") {
+    val expectedMap = Map(
+      "os" -> Set("centos7"),
+      "zone" -> Set("us-east-1a", "us-east-1b")
+    )
+    utils.parseConstraintString("os:centos7;zone:us-east-1a,us-east-1b") should be (expectedMap)
+  }
+
+  test("parse an empty constraint string correctly") {
+    utils.parseConstraintString("") shouldBe Map()
+  }
+
+  test("throw an exception when the input is malformed") {
+    an[IllegalArgumentException] should be thrownBy
+      utils.parseConstraintString("os;zone:us-east")
+  }
+
+  test("empty values for attributes' constraints matches all values") {
+    val constraintsStr = "os:"
+    val parsedConstraints = utils.parseConstraintString(constraintsStr)
+
+    parsedConstraints shouldBe Map("os" -> Set())
+
+    val zoneSet = Value.Set.newBuilder().addItem("us-east-1a").addItem("us-east-1b").build()
+    val noOsOffer = Map("zone" -> zoneSet)
+    val centosOffer = Map("os" -> Value.Text.newBuilder().setValue("centos").build())
+    val ubuntuOffer = Map("os" -> Value.Text.newBuilder().setValue("ubuntu").build())
+
+    utils.matchesAttributeRequirements(parsedConstraints, noOsOffer) shouldBe false
+    utils.matchesAttributeRequirements(parsedConstraints, centosOffer) shouldBe true
+    utils.matchesAttributeRequirements(parsedConstraints, ubuntuOffer) shouldBe true
+  }
+
+  test("subset match is performed for set attributes") {
+    val supersetConstraint = Map(
+      "os" -> Value.Text.newBuilder().setValue("ubuntu").build(),
+      "zone" -> Value.Set.newBuilder()
+        .addItem("us-east-1a")
+        .addItem("us-east-1b")
+        .addItem("us-east-1c")
+        .build())
+
+    val zoneConstraintStr = "os:;zone:us-east-1a,us-east-1c"
+    val parsedConstraints = utils.parseConstraintString(zoneConstraintStr)
+
+    utils.matchesAttributeRequirements(parsedConstraints, supersetConstraint) shouldBe true
+  }
+
+  test("less than equal match is performed on scalar attributes") {
+    val offerAttribs = Map("gpus" -> Value.Scalar.newBuilder().setValue(3).build())
+
+    val ltConstraint = utils.parseConstraintString("gpus:2")
+    val eqConstraint = utils.parseConstraintString("gpus:3")
+    val gtConstraint = utils.parseConstraintString("gpus:4")
+
+    utils.matchesAttributeRequirements(ltConstraint, offerAttribs) shouldBe true
+    utils.matchesAttributeRequirements(eqConstraint, offerAttribs) shouldBe true
+    utils.matchesAttributeRequirements(gtConstraint, offerAttribs) shouldBe false
+  }
+
+  test("contains match is performed for range attributes") {
+    val offerAttribs = Map("ports" -> Value.Range.newBuilder().setBegin(7000).setEnd(8000).build())
+    val ltConstraint = utils.parseConstraintString("ports:6000")
+    val eqConstraint = utils.parseConstraintString("ports:7500")
+    val gtConstraint = utils.parseConstraintString("ports:8002")
+    val multiConstraint = utils.parseConstraintString("ports:5000,7500,8300")
+
+    utils.matchesAttributeRequirements(ltConstraint, offerAttribs) shouldBe false
+    utils.matchesAttributeRequirements(eqConstraint, offerAttribs) shouldBe true
+    utils.matchesAttributeRequirements(gtConstraint, offerAttribs) shouldBe false
+    utils.matchesAttributeRequirements(multiConstraint, offerAttribs) shouldBe true
+  }
+
+  test("equality match is performed for text attributes") {
+    val offerAttribs = Map("os" -> Value.Text.newBuilder().setValue("centos7").build())
+
+    val trueConstraint = utils.parseConstraintString("os:centos7")
+    val falseConstraint = utils.parseConstraintString("os:ubuntu")
+
+    utils.matchesAttributeRequirements(trueConstraint, offerAttribs) shouldBe true
+    utils.matchesAttributeRequirements(falseConstraint, offerAttribs) shouldBe false
+  }
+
+  test("Port reservation is done correctly with user specified ports only") {
+    val conf = new SparkConf()
+    conf.set("spark.executor.port", "3000" )
+    conf.set(BLOCK_MANAGER_PORT, 4000)
+    val portResource = createTestPortResource((3000, 5000), Some("my_role"))
+
+    val (resourcesLeft, resourcesToBeUsed) = utils
+      .partitionPortResources(List(3000, 4000), List(portResource))
+    resourcesToBeUsed.length shouldBe 2
+
+    val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1}.toArray
+
+    portsToUse.length shouldBe 2
+    arePortsEqual(portsToUse, Array(3000L, 4000L)) shouldBe true
+
+    val portRangesToBeUsed = rangesResourcesToTuple(resourcesToBeUsed)
+
+    val expectedUSed = Array((3000L, 3000L), (4000L, 4000L))
+
+    arePortsEqual(portRangesToBeUsed.toArray, expectedUSed) shouldBe true
+  }
+
+  test("Port reservation is done correctly with some user specified ports (spark.executor.port)") {
+    val conf = new SparkConf()
+    conf.set("spark.executor.port", "3100" )
+    val portResource = createTestPortResource((3000, 5000), Some("my_role"))
+
+    val (resourcesLeft, resourcesToBeUsed) = utils
+      .partitionPortResources(List(3100), List(portResource))
+
+    val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1}
+
+    portsToUse.length shouldBe 1
+    portsToUse.contains(3100) shouldBe true
+  }
+
+  test("Port reservation is done correctly with all random ports") {
+    val conf = new SparkConf()
+    val portResource = createTestPortResource((3000L, 5000L), Some("my_role"))
+
+    val (resourcesLeft, resourcesToBeUsed) = utils
+      .partitionPortResources(List(), List(portResource))
+    val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1}
+
+    portsToUse.isEmpty shouldBe true
+  }
+
+  test("Port reservation is done correctly with user specified ports only - multiple ranges") {
+    val conf = new SparkConf()
+    conf.set("spark.executor.port", "2100" )
+    conf.set("spark.blockManager.port", "4000")
+    val portResourceList = List(createTestPortResource((3000, 5000), Some("my_role")),
+      createTestPortResource((2000, 2500), Some("other_role")))
+    val (resourcesLeft, resourcesToBeUsed) = utils
+      .partitionPortResources(List(2100, 4000), portResourceList)
+    val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1}
+
+    portsToUse.length shouldBe 2
+    val portsRangesLeft = rangesResourcesToTuple(resourcesLeft)
+    val portRangesToBeUsed = rangesResourcesToTuple(resourcesToBeUsed)
+
+    val expectedUsed = Array((2100L, 2100L), (4000L, 4000L))
+
+    arePortsEqual(portsToUse.toArray, Array(2100L, 4000L)) shouldBe true
+    arePortsEqual(portRangesToBeUsed.toArray, expectedUsed) shouldBe true
+  }
+
+  test("Port reservation is done correctly with all random ports - multiple ranges") {
+    val conf = new SparkConf()
+    val portResourceList = List(createTestPortResource((3000, 5000), Some("my_role")),
+      createTestPortResource((2000, 2500), Some("other_role")))
+    val (resourcesLeft, resourcesToBeUsed) = utils
+      .partitionPortResources(List(), portResourceList)
+    val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1}
+    portsToUse.isEmpty shouldBe true
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala
new file mode 100644
index 0000000..5a81bb3
--- /dev/null
+++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala
@@ -0,0 +1,36 @@
+/*
+ * 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.mesos
+
+import java.nio.ByteBuffer
+
+import org.apache.spark.SparkFunSuite
+
+class MesosTaskLaunchDataSuite extends SparkFunSuite {
+  test("serialize and deserialize data must be same") {
+    val serializedTask = ByteBuffer.allocate(40)
+    (Range(100, 110).map(serializedTask.putInt(_)))
+    serializedTask.rewind
+    val attemptNumber = 100
+    val byteString = MesosTaskLaunchData(serializedTask, attemptNumber).toByteString
+    serializedTask.rewind
+    val mesosTaskLaunchData = MesosTaskLaunchData.fromByteString(byteString)
+    assert(mesosTaskLaunchData.attemptNumber == attemptNumber)
+    assert(mesosTaskLaunchData.serializedTask.equals(serializedTask))
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala
new file mode 100644
index 0000000..7ebb294
--- /dev/null
+++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler.cluster.mesos
+
+import java.util.Collections
+
+import scala.collection.JavaConverters._
+
+import org.apache.mesos.Protos._
+import org.apache.mesos.Protos.Value.{Range => MesosRange, Ranges, Scalar}
+import org.apache.mesos.SchedulerDriver
+import org.mockito.{ArgumentCaptor, Matchers}
+import org.mockito.Mockito._
+
+object Utils {
+  def createOffer(
+      offerId: String,
+      slaveId: String,
+      mem: Int,
+      cpus: Int,
+      ports: Option[(Long, Long)] = None,
+      gpus: Int = 0): Offer = {
+    val builder = Offer.newBuilder()
+    builder.addResourcesBuilder()
+      .setName("mem")
+      .setType(Value.Type.SCALAR)
+      .setScalar(Scalar.newBuilder().setValue(mem))
+    builder.addResourcesBuilder()
+      .setName("cpus")
+      .setType(Value.Type.SCALAR)
+      .setScalar(Scalar.newBuilder().setValue(cpus))
+    ports.foreach { resourcePorts =>
+      builder.addResourcesBuilder()
+        .setName("ports")
+        .setType(Value.Type.RANGES)
+        .setRanges(Ranges.newBuilder().addRange(MesosRange.newBuilder()
+          .setBegin(resourcePorts._1).setEnd(resourcePorts._2).build()))
+    }
+    if (gpus > 0) {
+      builder.addResourcesBuilder()
+        .setName("gpus")
+        .setType(Value.Type.SCALAR)
+        .setScalar(Scalar.newBuilder().setValue(gpus))
+    }
+    builder.setId(createOfferId(offerId))
+      .setFrameworkId(FrameworkID.newBuilder()
+        .setValue("f1"))
+      .setSlaveId(SlaveID.newBuilder().setValue(slaveId))
+      .setHostname(s"host${slaveId}")
+      .build()
+  }
+
+  def verifyTaskLaunched(driver: SchedulerDriver, offerId: String): List[TaskInfo] = {
+    val captor = ArgumentCaptor.forClass(classOf[java.util.Collection[TaskInfo]])
+    verify(driver, times(1)).launchTasks(
+      Matchers.eq(Collections.singleton(createOfferId(offerId))),
+      captor.capture())
+    captor.getValue.asScala.toList
+  }
+
+  def createOfferId(offerId: String): OfferID = {
+    OfferID.newBuilder().setValue(offerId).build()
+  }
+
+  def createSlaveId(slaveId: String): SlaveID = {
+    SlaveID.newBuilder().setValue(slaveId).build()
+  }
+
+  def createExecutorId(executorId: String): ExecutorID = {
+    ExecutorID.newBuilder().setValue(executorId).build()
+  }
+
+  def createTaskId(taskId: String): TaskID = {
+    TaskID.newBuilder().setValue(taskId).build()
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml
new file mode 100644
index 0000000..04b51dc
--- /dev/null
+++ b/resource-managers/yarn/pom.xml
@@ -0,0 +1,215 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent_2.11</artifactId>
+    <version>2.2.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>spark-yarn_2.11</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project YARN</name>
+  <properties>
+    <sbt.project.name>yarn</sbt.project.name>
+    <jersey-1.version>1.9</jersey-1.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-network-yarn_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-tags_${scala.binary.version}</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-web-proxy</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+    </dependency>
+
+    <!-- Explicit listing of transitive deps that are shaded. Otherwise, odd compiler crashes. -->
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-plus</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-http</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlet</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlets</artifactId>
+    </dependency>
+    <!-- End of shaded deps. -->
+
+    <!--
+      SPARK-10059: Explicitly add JSP dependencies for tests since the MiniYARN cluster needs them.
+    -->
+    <dependency>
+      <groupId>org.eclipse.jetty.orbit</groupId>
+      <artifactId>javax.servlet.jsp</artifactId>
+      <version>2.2.0.v201112011158</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty.orbit</groupId>
+      <artifactId>javax.servlet.jsp.jstl</artifactId>
+      <version>1.2.0.v201105211821</version>
+      <scope>test</scope>
+    </dependency>
+
+     <!--
+    See SPARK-3710. hadoop-yarn-server-tests in Hadoop 2.2 fails to pull some needed
+    dependencies, so they need to be added manually for the tests to work.
+    -->
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-tests</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty</artifactId>
+      <version>6.1.26</version>
+      <exclusions>
+       <exclusion>
+        <groupId>org.mortbay.jetty</groupId>
+         <artifactId>servlet-api</artifactId>
+       </exclusion>
+      </exclusions>
+      <scope>test</scope>
+     </dependency>
+
+     <!--
+       Jersey 1 dependencies only required for YARN integration testing. Creating a YARN cluster
+       in the JVM requires starting a Jersey 1-based web application.
+     -->
+     <dependency>
+       <groupId>com.sun.jersey</groupId>
+       <artifactId>jersey-core</artifactId>
+       <scope>test</scope>
+       <version>${jersey-1.version}</version>
+     </dependency>
+     <dependency>
+       <groupId>com.sun.jersey</groupId>
+       <artifactId>jersey-json</artifactId>
+       <scope>test</scope>
+       <version>${jersey-1.version}</version>
+     </dependency>
+     <dependency>
+       <groupId>com.sun.jersey</groupId>
+       <artifactId>jersey-server</artifactId>
+       <scope>test</scope>
+       <version>${jersey-1.version}</version>
+     </dependency>
+     <dependency>
+       <groupId>com.sun.jersey.contribs</groupId>
+       <artifactId>jersey-guice</artifactId>
+       <scope>test</scope>
+       <version>${jersey-1.version}</version>
+     </dependency>
+
+    <!--
+     Testing Hive reflection needs hive on the test classpath only.
+     It doesn't need the spark hive modules, so the -Phive flag is not checked.
+      -->
+    <dependency>
+      <groupId>${hive.group}</groupId>
+      <artifactId>hive-exec</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>${hive.group}</groupId>
+      <artifactId>hive-metastore</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.thrift</groupId>
+      <artifactId>libthrift</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.thrift</groupId>
+      <artifactId>libfb303</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider b/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
new file mode 100644
index 0000000..22ead56
--- /dev/null
+++ b/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
@@ -0,0 +1,3 @@
+org.apache.spark.deploy.yarn.security.HDFSCredentialProvider
+org.apache.spark.deploy.yarn.security.HBaseCredentialProvider
+org.apache.spark.deploy.yarn.security.HiveCredentialProvider

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
new file mode 100644
index 0000000..6e8a1eb
--- /dev/null
+++ b/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
@@ -0,0 +1 @@
+org.apache.spark.scheduler.cluster.YarnClusterManager

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
new file mode 100644
index 0000000..0378ef4
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -0,0 +1,791 @@
+/*
+ * 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 java.io.{File, IOException}
+import java.lang.reflect.InvocationTargetException
+import java.net.{Socket, URI, URL}
+import java.util.concurrent.{TimeoutException, TimeUnit}
+
+import scala.collection.mutable.HashMap
+import scala.concurrent.Promise
+import scala.concurrent.duration.Duration
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+
+import org.apache.spark._
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.history.HistoryServer
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.deploy.yarn.security.{AMCredentialRenewer, ConfigurableCredentialManager}
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc._
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, YarnSchedulerBackend}
+import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
+import org.apache.spark.util._
+
+/**
+ * Common application master functionality for Spark on Yarn.
+ */
+private[spark] class ApplicationMaster(
+    args: ApplicationMasterArguments,
+    client: YarnRMClient)
+  extends Logging {
+
+  // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be
+  // optimal as more containers are available. Might need to handle this better.
+
+  private val sparkConf = new SparkConf()
+  private val yarnConf: YarnConfiguration = SparkHadoopUtil.get.newConfiguration(sparkConf)
+    .asInstanceOf[YarnConfiguration]
+  private val isClusterMode = args.userClass != null
+
+  // Default to twice the number of executors (twice the maximum number of executors if dynamic
+  // allocation is enabled), with a minimum of 3.
+
+  private val maxNumExecutorFailures = {
+    val effectiveNumExecutors =
+      if (Utils.isDynamicAllocationEnabled(sparkConf)) {
+        sparkConf.get(DYN_ALLOCATION_MAX_EXECUTORS)
+      } else {
+        sparkConf.get(EXECUTOR_INSTANCES).getOrElse(0)
+      }
+    // By default, effectiveNumExecutors is Int.MaxValue if dynamic allocation is enabled. We need
+    // avoid the integer overflow here.
+    val defaultMaxNumExecutorFailures = math.max(3,
+      if (effectiveNumExecutors > Int.MaxValue / 2) Int.MaxValue else (2 * effectiveNumExecutors))
+
+    sparkConf.get(MAX_EXECUTOR_FAILURES).getOrElse(defaultMaxNumExecutorFailures)
+  }
+
+  @volatile private var exitCode = 0
+  @volatile private var unregistered = false
+  @volatile private var finished = false
+  @volatile private var finalStatus = getDefaultFinalStatus
+  @volatile private var finalMsg: String = ""
+  @volatile private var userClassThread: Thread = _
+
+  @volatile private var reporterThread: Thread = _
+  @volatile private var allocator: YarnAllocator = _
+
+  // Lock for controlling the allocator (heartbeat) thread.
+  private val allocatorLock = new Object()
+
+  // Steady state heartbeat interval. We want to be reasonably responsive without causing too many
+  // requests to RM.
+  private val heartbeatInterval = {
+    // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses.
+    val expiryInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
+    math.max(0, math.min(expiryInterval / 2, sparkConf.get(RM_HEARTBEAT_INTERVAL)))
+  }
+
+  // Initial wait interval before allocator poll, to allow for quicker ramp up when executors are
+  // being requested.
+  private val initialAllocationInterval = math.min(heartbeatInterval,
+    sparkConf.get(INITIAL_HEARTBEAT_INTERVAL))
+
+  // Next wait interval before allocator poll.
+  private var nextAllocationInterval = initialAllocationInterval
+
+  private var rpcEnv: RpcEnv = null
+  private var amEndpoint: RpcEndpointRef = _
+
+  // In cluster mode, used to tell the AM when the user's SparkContext has been initialized.
+  private val sparkContextPromise = Promise[SparkContext]()
+
+  private var credentialRenewer: AMCredentialRenewer = _
+
+  // Load the list of localized files set by the client. This is used when launching executors,
+  // and is loaded here so that these configs don't pollute the Web UI's environment page in
+  // cluster mode.
+  private val localResources = {
+    logInfo("Preparing Local resources")
+    val resources = HashMap[String, LocalResource]()
+
+    def setupDistributedCache(
+        file: String,
+        rtype: LocalResourceType,
+        timestamp: String,
+        size: String,
+        vis: String): Unit = {
+      val uri = new URI(file)
+      val amJarRsrc = Records.newRecord(classOf[LocalResource])
+      amJarRsrc.setType(rtype)
+      amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis))
+      amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri))
+      amJarRsrc.setTimestamp(timestamp.toLong)
+      amJarRsrc.setSize(size.toLong)
+
+      val fileName = Option(uri.getFragment()).getOrElse(new Path(uri).getName())
+      resources(fileName) = amJarRsrc
+    }
+
+    val distFiles = sparkConf.get(CACHED_FILES)
+    val fileSizes = sparkConf.get(CACHED_FILES_SIZES)
+    val timeStamps = sparkConf.get(CACHED_FILES_TIMESTAMPS)
+    val visibilities = sparkConf.get(CACHED_FILES_VISIBILITIES)
+    val resTypes = sparkConf.get(CACHED_FILES_TYPES)
+
+    for (i <- 0 to distFiles.size - 1) {
+      val resType = LocalResourceType.valueOf(resTypes(i))
+      setupDistributedCache(distFiles(i), resType, timeStamps(i).toString, fileSizes(i).toString,
+      visibilities(i))
+    }
+
+    // Distribute the conf archive to executors.
+    sparkConf.get(CACHED_CONF_ARCHIVE).foreach { path =>
+      val uri = new URI(path)
+      val fs = FileSystem.get(uri, yarnConf)
+      val status = fs.getFileStatus(new Path(uri))
+      // SPARK-16080: Make sure to use the correct name for the destination when distributing the
+      // conf archive to executors.
+      val destUri = new URI(uri.getScheme(), uri.getRawSchemeSpecificPart(),
+        Client.LOCALIZED_CONF_DIR)
+      setupDistributedCache(destUri.toString(), LocalResourceType.ARCHIVE,
+        status.getModificationTime().toString, status.getLen.toString,
+        LocalResourceVisibility.PRIVATE.name())
+    }
+
+    // Clean up the configuration so it doesn't show up in the Web UI (since it's really noisy).
+    CACHE_CONFIGS.foreach { e =>
+      sparkConf.remove(e)
+      sys.props.remove(e.key)
+    }
+
+    resources.toMap
+  }
+
+  def getAttemptId(): ApplicationAttemptId = {
+    client.getAttemptId()
+  }
+
+  final def run(): Int = {
+    try {
+      val appAttemptId = client.getAttemptId()
+
+      var attemptID: Option[String] = None
+
+      if (isClusterMode) {
+        // Set the web ui port to be ephemeral for yarn so we don't conflict with
+        // other spark processes running on the same box
+        System.setProperty("spark.ui.port", "0")
+
+        // Set the master and deploy mode property to match the requested mode.
+        System.setProperty("spark.master", "yarn")
+        System.setProperty("spark.submit.deployMode", "cluster")
+
+        // Set this internal configuration if it is running on cluster mode, this
+        // configuration will be checked in SparkContext to avoid misuse of yarn cluster mode.
+        System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString())
+
+        attemptID = Option(appAttemptId.getAttemptId.toString)
+      }
+
+      new CallerContext(
+        "APPMASTER", sparkConf.get(APP_CALLER_CONTEXT),
+        Option(appAttemptId.getApplicationId.toString), attemptID).setCurrentContext()
+
+      logInfo("ApplicationAttemptId: " + appAttemptId)
+
+      val fs = FileSystem.get(yarnConf)
+
+      // This shutdown hook should run *after* the SparkContext is shut down.
+      val priority = ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY - 1
+      ShutdownHookManager.addShutdownHook(priority) { () =>
+        val maxAppAttempts = client.getMaxRegAttempts(sparkConf, yarnConf)
+        val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts
+
+        if (!finished) {
+          // The default state of ApplicationMaster is failed if it is invoked by shut down hook.
+          // This behavior is different compared to 1.x version.
+          // If user application is exited ahead of time by calling System.exit(N), here mark
+          // this application as failed with EXIT_EARLY. For a good shutdown, user shouldn't call
+          // System.exit(0) to terminate the application.
+          finish(finalStatus,
+            ApplicationMaster.EXIT_EARLY,
+            "Shutdown hook called before final status was reported.")
+        }
+
+        if (!unregistered) {
+          // we only want to unregister if we don't want the RM to retry
+          if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) {
+            unregister(finalStatus, finalMsg)
+            cleanupStagingDir(fs)
+          }
+        }
+      }
+
+      // Call this to force generation of secret so it gets populated into the
+      // Hadoop UGI. This has to happen before the startUserApplication which does a
+      // doAs in order for the credentials to be passed on to the executor containers.
+      val securityMgr = new SecurityManager(sparkConf)
+
+      // If the credentials file config is present, we must periodically renew tokens. So create
+      // a new AMDelegationTokenRenewer
+      if (sparkConf.contains(CREDENTIALS_FILE_PATH.key)) {
+        // If a principal and keytab have been set, use that to create new credentials for executors
+        // periodically
+        credentialRenewer =
+          new ConfigurableCredentialManager(sparkConf, yarnConf).credentialRenewer()
+        credentialRenewer.scheduleLoginFromKeytab()
+      }
+
+      if (isClusterMode) {
+        runDriver(securityMgr)
+      } else {
+        runExecutorLauncher(securityMgr)
+      }
+    } catch {
+      case e: Exception =>
+        // catch everything else if not specifically handled
+        logError("Uncaught exception: ", e)
+        finish(FinalApplicationStatus.FAILED,
+          ApplicationMaster.EXIT_UNCAUGHT_EXCEPTION,
+          "Uncaught exception: " + e)
+    }
+    exitCode
+  }
+
+  /**
+   * Set the default final application status for client mode to UNDEFINED to handle
+   * if YARN HA restarts the application so that it properly retries. Set the final
+   * status to SUCCEEDED in cluster mode to handle if the user calls System.exit
+   * from the application code.
+   */
+  final def getDefaultFinalStatus(): FinalApplicationStatus = {
+    if (isClusterMode) {
+      FinalApplicationStatus.FAILED
+    } else {
+      FinalApplicationStatus.UNDEFINED
+    }
+  }
+
+  /**
+   * unregister is used to completely unregister the application from the ResourceManager.
+   * This means the ResourceManager will not retry the application attempt on your behalf if
+   * a failure occurred.
+   */
+  final def unregister(status: FinalApplicationStatus, diagnostics: String = null): Unit = {
+    synchronized {
+      if (!unregistered) {
+        logInfo(s"Unregistering ApplicationMaster with $status" +
+          Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse(""))
+        unregistered = true
+        client.unregister(status, Option(diagnostics).getOrElse(""))
+      }
+    }
+  }
+
+  final def finish(status: FinalApplicationStatus, code: Int, msg: String = null): Unit = {
+    synchronized {
+      if (!finished) {
+        val inShutdown = ShutdownHookManager.inShutdown()
+        logInfo(s"Final app status: $status, exitCode: $code" +
+          Option(msg).map(msg => s", (reason: $msg)").getOrElse(""))
+        exitCode = code
+        finalStatus = status
+        finalMsg = msg
+        finished = true
+        if (!inShutdown && Thread.currentThread() != reporterThread && reporterThread != null) {
+          logDebug("shutting down reporter thread")
+          reporterThread.interrupt()
+        }
+        if (!inShutdown && Thread.currentThread() != userClassThread && userClassThread != null) {
+          logDebug("shutting down user thread")
+          userClassThread.interrupt()
+        }
+        if (!inShutdown && credentialRenewer != null) {
+          credentialRenewer.stop()
+          credentialRenewer = null
+        }
+      }
+    }
+  }
+
+  private def sparkContextInitialized(sc: SparkContext) = {
+    sparkContextPromise.success(sc)
+  }
+
+  private def registerAM(
+      _sparkConf: SparkConf,
+      _rpcEnv: RpcEnv,
+      driverRef: RpcEndpointRef,
+      uiAddress: String,
+      securityMgr: SecurityManager) = {
+    val appId = client.getAttemptId().getApplicationId().toString()
+    val attemptId = client.getAttemptId().getAttemptId().toString()
+    val historyAddress =
+      _sparkConf.get(HISTORY_SERVER_ADDRESS)
+        .map { text => SparkHadoopUtil.get.substituteHadoopVariables(text, yarnConf) }
+        .map { address => s"${address}${HistoryServer.UI_PATH_PREFIX}/${appId}/${attemptId}" }
+        .getOrElse("")
+
+    val driverUrl = RpcEndpointAddress(
+      _sparkConf.get("spark.driver.host"),
+      _sparkConf.get("spark.driver.port").toInt,
+      CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+    // Before we initialize the allocator, let's log the information about how executors will
+    // be run up front, to avoid printing this out for every single executor being launched.
+    // Use placeholders for information that changes such as executor IDs.
+    logInfo {
+      val executorMemory = sparkConf.get(EXECUTOR_MEMORY).toInt
+      val executorCores = sparkConf.get(EXECUTOR_CORES)
+      val dummyRunner = new ExecutorRunnable(None, yarnConf, sparkConf, driverUrl, "<executorId>",
+        "<hostname>", executorMemory, executorCores, appId, securityMgr, localResources)
+      dummyRunner.launchContextDebugInfo()
+    }
+
+    allocator = client.register(driverUrl,
+      driverRef,
+      yarnConf,
+      _sparkConf,
+      uiAddress,
+      historyAddress,
+      securityMgr,
+      localResources)
+
+    allocator.allocateResources()
+    reporterThread = launchReporterThread()
+  }
+
+  /**
+   * Create an [[RpcEndpoint]] that communicates with the driver.
+   *
+   * In cluster mode, the AM and the driver belong to same process
+   * so the AMEndpoint need not monitor lifecycle of the driver.
+   *
+   * @return A reference to the driver's RPC endpoint.
+   */
+  private def runAMEndpoint(
+      host: String,
+      port: String,
+      isClusterMode: Boolean): RpcEndpointRef = {
+    val driverEndpoint = rpcEnv.setupEndpointRef(
+      RpcAddress(host, port.toInt),
+      YarnSchedulerBackend.ENDPOINT_NAME)
+    amEndpoint =
+      rpcEnv.setupEndpoint("YarnAM", new AMEndpoint(rpcEnv, driverEndpoint, isClusterMode))
+    driverEndpoint
+  }
+
+  private def runDriver(securityMgr: SecurityManager): Unit = {
+    addAmIpFilter()
+    userClassThread = startUserApplication()
+
+    // This a bit hacky, but we need to wait until the spark.driver.port property has
+    // been set by the Thread executing the user class.
+    logInfo("Waiting for spark context initialization...")
+    val totalWaitTime = sparkConf.get(AM_MAX_WAIT_TIME)
+    try {
+      val sc = ThreadUtils.awaitResult(sparkContextPromise.future,
+        Duration(totalWaitTime, TimeUnit.MILLISECONDS))
+      if (sc != null) {
+        rpcEnv = sc.env.rpcEnv
+        val driverRef = runAMEndpoint(
+          sc.getConf.get("spark.driver.host"),
+          sc.getConf.get("spark.driver.port"),
+          isClusterMode = true)
+        registerAM(sc.getConf, rpcEnv, driverRef, sc.ui.map(_.webUrl).getOrElse(""),
+          securityMgr)
+      } else {
+        // Sanity check; should never happen in normal operation, since sc should only be null
+        // if the user app did not create a SparkContext.
+        if (!finished) {
+          throw new IllegalStateException("SparkContext is null but app is still running!")
+        }
+      }
+      userClassThread.join()
+    } catch {
+      case e: SparkException if e.getCause().isInstanceOf[TimeoutException] =>
+        logError(
+          s"SparkContext did not initialize after waiting for $totalWaitTime ms. " +
+           "Please check earlier log output for errors. Failing the application.")
+        finish(FinalApplicationStatus.FAILED,
+          ApplicationMaster.EXIT_SC_NOT_INITED,
+          "Timed out waiting for SparkContext.")
+    }
+  }
+
+  private def runExecutorLauncher(securityMgr: SecurityManager): Unit = {
+    val port = sparkConf.get(AM_PORT)
+    rpcEnv = RpcEnv.create("sparkYarnAM", Utils.localHostName, port, sparkConf, securityMgr,
+      clientMode = true)
+    val driverRef = waitForSparkDriver()
+    addAmIpFilter()
+    registerAM(sparkConf, rpcEnv, driverRef, sparkConf.get("spark.driver.appUIAddress", ""),
+      securityMgr)
+
+    // In client mode the actor will stop the reporter thread.
+    reporterThread.join()
+  }
+
+  private def launchReporterThread(): Thread = {
+    // The number of failures in a row until Reporter thread give up
+    val reporterMaxFailures = sparkConf.get(MAX_REPORTER_THREAD_FAILURES)
+
+    val t = new Thread {
+      override def run() {
+        var failureCount = 0
+        while (!finished) {
+          try {
+            if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) {
+              finish(FinalApplicationStatus.FAILED,
+                ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES,
+                s"Max number of executor failures ($maxNumExecutorFailures) reached")
+            } else {
+              logDebug("Sending progress")
+              allocator.allocateResources()
+            }
+            failureCount = 0
+          } catch {
+            case i: InterruptedException =>
+            case e: Throwable =>
+              failureCount += 1
+              // this exception was introduced in hadoop 2.4 and this code would not compile
+              // with earlier versions if we refer it directly.
+              if ("org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException" ==
+                e.getClass().getName()) {
+                logError("Exception from Reporter thread.", e)
+                finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_REPORTER_FAILURE,
+                  e.getMessage)
+              } else if (!NonFatal(e) || failureCount >= reporterMaxFailures) {
+                finish(FinalApplicationStatus.FAILED,
+                  ApplicationMaster.EXIT_REPORTER_FAILURE, "Exception was thrown " +
+                    s"$failureCount time(s) from Reporter thread.")
+              } else {
+                logWarning(s"Reporter thread fails $failureCount time(s) in a row.", e)
+              }
+          }
+          try {
+            val numPendingAllocate = allocator.getPendingAllocate.size
+            var sleepStart = 0L
+            var sleepInterval = 200L // ms
+            allocatorLock.synchronized {
+              sleepInterval =
+                if (numPendingAllocate > 0 || allocator.getNumPendingLossReasonRequests > 0) {
+                  val currentAllocationInterval =
+                    math.min(heartbeatInterval, nextAllocationInterval)
+                  nextAllocationInterval = currentAllocationInterval * 2 // avoid overflow
+                  currentAllocationInterval
+                } else {
+                  nextAllocationInterval = initialAllocationInterval
+                  heartbeatInterval
+                }
+              sleepStart = System.currentTimeMillis()
+              allocatorLock.wait(sleepInterval)
+            }
+            val sleepDuration = System.currentTimeMillis() - sleepStart
+            if (sleepDuration < sleepInterval) {
+              // log when sleep is interrupted
+              logDebug(s"Number of pending allocations is $numPendingAllocate. " +
+                  s"Slept for $sleepDuration/$sleepInterval ms.")
+              // if sleep was less than the minimum interval, sleep for the rest of it
+              val toSleep = math.max(0, initialAllocationInterval - sleepDuration)
+              if (toSleep > 0) {
+                logDebug(s"Going back to sleep for $toSleep ms")
+                // use Thread.sleep instead of allocatorLock.wait. there is no need to be woken up
+                // by the methods that signal allocatorLock because this is just finishing the min
+                // sleep interval, which should happen even if this is signalled again.
+                Thread.sleep(toSleep)
+              }
+            } else {
+              logDebug(s"Number of pending allocations is $numPendingAllocate. " +
+                  s"Slept for $sleepDuration/$sleepInterval.")
+            }
+          } catch {
+            case e: InterruptedException =>
+          }
+        }
+      }
+    }
+    // setting to daemon status, though this is usually not a good idea.
+    t.setDaemon(true)
+    t.setName("Reporter")
+    t.start()
+    logInfo(s"Started progress reporter thread with (heartbeat : $heartbeatInterval, " +
+            s"initial allocation : $initialAllocationInterval) intervals")
+    t
+  }
+
+  /**
+   * Clean up the staging directory.
+   */
+  private def cleanupStagingDir(fs: FileSystem) {
+    var stagingDirPath: Path = null
+    try {
+      val preserveFiles = sparkConf.get(PRESERVE_STAGING_FILES)
+      if (!preserveFiles) {
+        stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR"))
+        if (stagingDirPath == null) {
+          logError("Staging directory is null")
+          return
+        }
+        logInfo("Deleting staging directory " + stagingDirPath)
+        fs.delete(stagingDirPath, true)
+      }
+    } catch {
+      case ioe: IOException =>
+        logError("Failed to cleanup staging dir " + stagingDirPath, ioe)
+    }
+  }
+
+  private def waitForSparkDriver(): RpcEndpointRef = {
+    logInfo("Waiting for Spark driver to be reachable.")
+    var driverUp = false
+    val hostport = args.userArgs(0)
+    val (driverHost, driverPort) = Utils.parseHostPort(hostport)
+
+    // Spark driver should already be up since it launched us, but we don't want to
+    // wait forever, so wait 100 seconds max to match the cluster mode setting.
+    val totalWaitTimeMs = sparkConf.get(AM_MAX_WAIT_TIME)
+    val deadline = System.currentTimeMillis + totalWaitTimeMs
+
+    while (!driverUp && !finished && System.currentTimeMillis < deadline) {
+      try {
+        val socket = new Socket(driverHost, driverPort)
+        socket.close()
+        logInfo("Driver now available: %s:%s".format(driverHost, driverPort))
+        driverUp = true
+      } catch {
+        case e: Exception =>
+          logError("Failed to connect to driver at %s:%s, retrying ...".
+            format(driverHost, driverPort))
+          Thread.sleep(100L)
+      }
+    }
+
+    if (!driverUp) {
+      throw new SparkException("Failed to connect to driver!")
+    }
+
+    sparkConf.set("spark.driver.host", driverHost)
+    sparkConf.set("spark.driver.port", driverPort.toString)
+
+    runAMEndpoint(driverHost, driverPort.toString, isClusterMode = false)
+  }
+
+  /** Add the Yarn IP filter that is required for properly securing the UI. */
+  private def addAmIpFilter() = {
+    val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV)
+    val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter"
+    val params = client.getAmIpFilterParams(yarnConf, proxyBase)
+    if (isClusterMode) {
+      System.setProperty("spark.ui.filters", amFilter)
+      params.foreach { case (k, v) => System.setProperty(s"spark.$amFilter.param.$k", v) }
+    } else {
+      amEndpoint.send(AddWebUIFilter(amFilter, params.toMap, proxyBase))
+    }
+  }
+
+  /**
+   * Start the user class, which contains the spark driver, in a separate Thread.
+   * If the main routine exits cleanly or exits with System.exit(N) for any N
+   * we assume it was successful, for all other cases we assume failure.
+   *
+   * Returns the user thread that was started.
+   */
+  private def startUserApplication(): Thread = {
+    logInfo("Starting the user application in a separate Thread")
+
+    val classpath = Client.getUserClasspath(sparkConf)
+    val urls = classpath.map { entry =>
+      new URL("file:" + new File(entry.getPath()).getAbsolutePath())
+    }
+    val userClassLoader =
+      if (Client.isUserClassPathFirst(sparkConf, isDriver = true)) {
+        new ChildFirstURLClassLoader(urls, Utils.getContextOrSparkClassLoader)
+      } else {
+        new MutableURLClassLoader(urls, Utils.getContextOrSparkClassLoader)
+      }
+
+    var userArgs = args.userArgs
+    if (args.primaryPyFile != null && args.primaryPyFile.endsWith(".py")) {
+      // When running pyspark, the app is run using PythonRunner. The second argument is the list
+      // of files to add to PYTHONPATH, which Client.scala already handles, so it's empty.
+      userArgs = Seq(args.primaryPyFile, "") ++ userArgs
+    }
+    if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) {
+      // TODO(davies): add R dependencies here
+    }
+    val mainMethod = userClassLoader.loadClass(args.userClass)
+      .getMethod("main", classOf[Array[String]])
+
+    val userThread = new Thread {
+      override def run() {
+        try {
+          mainMethod.invoke(null, userArgs.toArray)
+          finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS)
+          logDebug("Done running users class")
+        } catch {
+          case e: InvocationTargetException =>
+            e.getCause match {
+              case _: InterruptedException =>
+                // Reporter thread can interrupt to stop user class
+              case SparkUserAppException(exitCode) =>
+                val msg = s"User application exited with status $exitCode"
+                logError(msg)
+                finish(FinalApplicationStatus.FAILED, exitCode, msg)
+              case cause: Throwable =>
+                logError("User class threw exception: " + cause, cause)
+                finish(FinalApplicationStatus.FAILED,
+                  ApplicationMaster.EXIT_EXCEPTION_USER_CLASS,
+                  "User class threw exception: " + cause)
+            }
+            sparkContextPromise.tryFailure(e.getCause())
+        } finally {
+          // Notify the thread waiting for the SparkContext, in case the application did not
+          // instantiate one. This will do nothing when the user code instantiates a SparkContext
+          // (with the correct master), or when the user code throws an exception (due to the
+          // tryFailure above).
+          sparkContextPromise.trySuccess(null)
+        }
+      }
+    }
+    userThread.setContextClassLoader(userClassLoader)
+    userThread.setName("Driver")
+    userThread.start()
+    userThread
+  }
+
+  private def resetAllocatorInterval(): Unit = allocatorLock.synchronized {
+    nextAllocationInterval = initialAllocationInterval
+    allocatorLock.notifyAll()
+  }
+
+  /**
+   * An [[RpcEndpoint]] that communicates with the driver's scheduler backend.
+   */
+  private class AMEndpoint(
+      override val rpcEnv: RpcEnv, driver: RpcEndpointRef, isClusterMode: Boolean)
+    extends RpcEndpoint with Logging {
+
+    override def onStart(): Unit = {
+      driver.send(RegisterClusterManager(self))
+    }
+
+    override def receive: PartialFunction[Any, Unit] = {
+      case x: AddWebUIFilter =>
+        logInfo(s"Add WebUI Filter. $x")
+        driver.send(x)
+    }
+
+    override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
+      case RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount) =>
+        Option(allocator) match {
+          case Some(a) =>
+            if (a.requestTotalExecutorsWithPreferredLocalities(requestedTotal,
+              localityAwareTasks, hostToLocalTaskCount)) {
+              resetAllocatorInterval()
+            }
+            context.reply(true)
+
+          case None =>
+            logWarning("Container allocator is not ready to request executors yet.")
+            context.reply(false)
+        }
+
+      case KillExecutors(executorIds) =>
+        logInfo(s"Driver requested to kill executor(s) ${executorIds.mkString(", ")}.")
+        Option(allocator) match {
+          case Some(a) => executorIds.foreach(a.killExecutor)
+          case None => logWarning("Container allocator is not ready to kill executors yet.")
+        }
+        context.reply(true)
+
+      case GetExecutorLossReason(eid) =>
+        Option(allocator) match {
+          case Some(a) =>
+            a.enqueueGetLossReasonRequest(eid, context)
+            resetAllocatorInterval()
+          case None =>
+            logWarning("Container allocator is not ready to find executor loss reasons yet.")
+        }
+    }
+
+    override def onDisconnected(remoteAddress: RpcAddress): Unit = {
+      // In cluster mode, do not rely on the disassociated event to exit
+      // This avoids potentially reporting incorrect exit codes if the driver fails
+      if (!isClusterMode) {
+        logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress")
+        finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS)
+      }
+    }
+  }
+
+}
+
+object ApplicationMaster extends Logging {
+
+  // exit codes for different causes, no reason behind the values
+  private val EXIT_SUCCESS = 0
+  private val EXIT_UNCAUGHT_EXCEPTION = 10
+  private val EXIT_MAX_EXECUTOR_FAILURES = 11
+  private val EXIT_REPORTER_FAILURE = 12
+  private val EXIT_SC_NOT_INITED = 13
+  private val EXIT_SECURITY = 14
+  private val EXIT_EXCEPTION_USER_CLASS = 15
+  private val EXIT_EARLY = 16
+
+  private var master: ApplicationMaster = _
+
+  def main(args: Array[String]): Unit = {
+    SignalUtils.registerLogger(log)
+    val amArgs = new ApplicationMasterArguments(args)
+
+    // Load the properties file with the Spark configuration and set entries as system properties,
+    // so that user code run inside the AM also has access to them.
+    // Note: we must do this before SparkHadoopUtil instantiated
+    if (amArgs.propertiesFile != null) {
+      Utils.getPropertiesFromFile(amArgs.propertiesFile).foreach { case (k, v) =>
+        sys.props(k) = v
+      }
+    }
+    SparkHadoopUtil.get.runAsSparkUser { () =>
+      master = new ApplicationMaster(amArgs, new YarnRMClient)
+      System.exit(master.run())
+    }
+  }
+
+  private[spark] def sparkContextInitialized(sc: SparkContext): Unit = {
+    master.sparkContextInitialized(sc)
+  }
+
+  private[spark] def getAttemptId(): ApplicationAttemptId = {
+    master.getAttemptId
+  }
+
+}
+
+/**
+ * This object does not provide any special functionality. It exists so that it's easy to tell
+ * apart the client-mode AM from the cluster-mode AM when using tools such as ps or jps.
+ */
+object ExecutorLauncher {
+
+  def main(args: Array[String]): Unit = {
+    ApplicationMaster.main(args)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
new file mode 100644
index 0000000..5cdec87
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
@@ -0,0 +1,105 @@
+/*
+ * 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 scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.util.{IntParam, MemoryParam}
+
+class ApplicationMasterArguments(val args: Array[String]) {
+  var userJar: String = null
+  var userClass: String = null
+  var primaryPyFile: String = null
+  var primaryRFile: String = null
+  var userArgs: Seq[String] = Nil
+  var propertiesFile: String = null
+
+  parseArgs(args.toList)
+
+  private def parseArgs(inputArgs: List[String]): Unit = {
+    val userArgsBuffer = new ArrayBuffer[String]()
+
+    var args = inputArgs
+
+    while (!args.isEmpty) {
+      // --num-workers, --worker-memory, and --worker-cores are deprecated since 1.0,
+      // the properties with executor in their names are preferred.
+      args match {
+        case ("--jar") :: value :: tail =>
+          userJar = value
+          args = tail
+
+        case ("--class") :: value :: tail =>
+          userClass = value
+          args = tail
+
+        case ("--primary-py-file") :: value :: tail =>
+          primaryPyFile = value
+          args = tail
+
+        case ("--primary-r-file") :: value :: tail =>
+          primaryRFile = value
+          args = tail
+
+        case ("--arg") :: value :: tail =>
+          userArgsBuffer += value
+          args = tail
+
+        case ("--properties-file") :: value :: tail =>
+          propertiesFile = value
+          args = tail
+
+        case _ =>
+          printUsageAndExit(1, args)
+      }
+    }
+
+    if (primaryPyFile != null && primaryRFile != null) {
+      // scalastyle:off println
+      System.err.println("Cannot have primary-py-file and primary-r-file at the same time")
+      // scalastyle:on println
+      System.exit(-1)
+    }
+
+    userArgs = userArgsBuffer.toList
+  }
+
+  def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
+    // scalastyle:off println
+    if (unknownParam != null) {
+      System.err.println("Unknown/unsupported param " + unknownParam)
+    }
+    System.err.println("""
+      |Usage: org.apache.spark.deploy.yarn.ApplicationMaster [options]
+      |Options:
+      |  --jar JAR_PATH       Path to your application's JAR file
+      |  --class CLASS_NAME   Name of your application's main class
+      |  --primary-py-file    A main Python file
+      |  --primary-r-file     A main R file
+      |  --arg ARG            Argument to be passed to your application's main class.
+      |                       Multiple invocations are possible, each will be passed in order.
+      |  --properties-file FILE Path to a custom Spark properties file.
+      """.stripMargin)
+    // scalastyle:on println
+    System.exit(exitCode)
+  }
+}
+
+object ApplicationMasterArguments {
+  val DEFAULT_NUMBER_EXECUTORS = 2
+}


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


[08/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala
new file mode 100644
index 0000000..9c3b18e
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala
@@ -0,0 +1,241 @@
+/*
+ * 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 java.io.{File, FileOutputStream, OutputStreamWriter}
+import java.nio.charset.StandardCharsets
+import java.util.Properties
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import com.google.common.io.Files
+import org.apache.commons.lang3.SerializationUtils
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.server.MiniYARNCluster
+import org.scalatest.{BeforeAndAfterAll, Matchers}
+import org.scalatest.concurrent.Eventually._
+
+import org.apache.spark._
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.launcher._
+import org.apache.spark.util.Utils
+
+abstract class BaseYarnClusterSuite
+  extends SparkFunSuite with BeforeAndAfterAll with Matchers with Logging {
+
+  // log4j configuration for the YARN containers, so that their output is collected
+  // by YARN instead of trying to overwrite unit-tests.log.
+  protected val LOG4J_CONF = """
+    |log4j.rootCategory=DEBUG, console
+    |log4j.appender.console=org.apache.log4j.ConsoleAppender
+    |log4j.appender.console.target=System.err
+    |log4j.appender.console.layout=org.apache.log4j.PatternLayout
+    |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n
+    |log4j.logger.org.apache.hadoop=WARN
+    |log4j.logger.org.eclipse.jetty=WARN
+    |log4j.logger.org.mortbay=WARN
+    |log4j.logger.org.spark_project.jetty=WARN
+    """.stripMargin
+
+  private var yarnCluster: MiniYARNCluster = _
+  protected var tempDir: File = _
+  private var fakeSparkJar: File = _
+  protected var hadoopConfDir: File = _
+  private var logConfDir: File = _
+
+  var oldSystemProperties: Properties = null
+
+  def newYarnConfig(): YarnConfiguration
+
+  override def beforeAll() {
+    super.beforeAll()
+    oldSystemProperties = SerializationUtils.clone(System.getProperties)
+
+    tempDir = Utils.createTempDir()
+    logConfDir = new File(tempDir, "log4j")
+    logConfDir.mkdir()
+    System.setProperty("SPARK_YARN_MODE", "true")
+
+    val logConfFile = new File(logConfDir, "log4j.properties")
+    Files.write(LOG4J_CONF, logConfFile, StandardCharsets.UTF_8)
+
+    // Disable the disk utilization check to avoid the test hanging when people's disks are
+    // getting full.
+    val yarnConf = newYarnConfig()
+    yarnConf.set("yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage",
+      "100.0")
+
+    yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1)
+    yarnCluster.init(yarnConf)
+    yarnCluster.start()
+
+    // There's a race in MiniYARNCluster in which start() may return before the RM has updated
+    // its address in the configuration. You can see this in the logs by noticing that when
+    // MiniYARNCluster prints the address, it still has port "0" assigned, although later the
+    // test works sometimes:
+    //
+    //    INFO MiniYARNCluster: MiniYARN ResourceManager address: blah:0
+    //
+    // That log message prints the contents of the RM_ADDRESS config variable. If you check it
+    // later on, it looks something like this:
+    //
+    //    INFO YarnClusterSuite: RM address in configuration is blah:42631
+    //
+    // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't
+    // done so in a timely manner (defined to be 10 seconds).
+    val config = yarnCluster.getConfig()
+    val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10)
+    while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") {
+      if (System.currentTimeMillis() > deadline) {
+        throw new IllegalStateException("Timed out waiting for RM to come up.")
+      }
+      logDebug("RM address still not set in configuration, waiting...")
+      TimeUnit.MILLISECONDS.sleep(100)
+    }
+
+    logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}")
+
+    fakeSparkJar = File.createTempFile("sparkJar", null, tempDir)
+    hadoopConfDir = new File(tempDir, Client.LOCALIZED_CONF_DIR)
+    assert(hadoopConfDir.mkdir())
+    File.createTempFile("token", ".txt", hadoopConfDir)
+  }
+
+  override def afterAll() {
+    try {
+      yarnCluster.stop()
+    } finally {
+      System.setProperties(oldSystemProperties)
+      super.afterAll()
+    }
+  }
+
+  protected def runSpark(
+      clientMode: Boolean,
+      klass: String,
+      appArgs: Seq[String] = Nil,
+      sparkArgs: Seq[(String, String)] = Nil,
+      extraClassPath: Seq[String] = Nil,
+      extraJars: Seq[String] = Nil,
+      extraConf: Map[String, String] = Map(),
+      extraEnv: Map[String, String] = Map()): SparkAppHandle.State = {
+    val deployMode = if (clientMode) "client" else "cluster"
+    val propsFile = createConfFile(extraClassPath = extraClassPath, extraConf = extraConf)
+    val env = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath()) ++ extraEnv
+
+    val launcher = new SparkLauncher(env.asJava)
+    if (klass.endsWith(".py")) {
+      launcher.setAppResource(klass)
+    } else {
+      launcher.setMainClass(klass)
+      launcher.setAppResource(fakeSparkJar.getAbsolutePath())
+    }
+    launcher.setSparkHome(sys.props("spark.test.home"))
+      .setMaster("yarn")
+      .setDeployMode(deployMode)
+      .setConf("spark.executor.instances", "1")
+      .setPropertiesFile(propsFile)
+      .addAppArgs(appArgs.toArray: _*)
+
+    sparkArgs.foreach { case (name, value) =>
+      if (value != null) {
+        launcher.addSparkArg(name, value)
+      } else {
+        launcher.addSparkArg(name)
+      }
+    }
+    extraJars.foreach(launcher.addJar)
+
+    val handle = launcher.startApplication()
+    try {
+      eventually(timeout(2 minutes), interval(1 second)) {
+        assert(handle.getState().isFinal())
+      }
+    } finally {
+      handle.kill()
+    }
+
+    handle.getState()
+  }
+
+  /**
+   * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide
+   * any sort of error when the job process finishes successfully, but the job itself fails. So
+   * the tests enforce that something is written to a file after everything is ok to indicate
+   * that the job succeeded.
+   */
+  protected def checkResult(finalState: SparkAppHandle.State, result: File): Unit = {
+    checkResult(finalState, result, "success")
+  }
+
+  protected def checkResult(
+      finalState: SparkAppHandle.State,
+      result: File,
+      expected: String): Unit = {
+    finalState should be (SparkAppHandle.State.FINISHED)
+    val resultString = Files.toString(result, StandardCharsets.UTF_8)
+    resultString should be (expected)
+  }
+
+  protected def mainClassName(klass: Class[_]): String = {
+    klass.getName().stripSuffix("$")
+  }
+
+  protected def createConfFile(
+      extraClassPath: Seq[String] = Nil,
+      extraConf: Map[String, String] = Map()): String = {
+    val props = new Properties()
+    props.put(SPARK_JARS.key, "local:" + fakeSparkJar.getAbsolutePath())
+
+    val testClasspath = new TestClasspathBuilder()
+      .buildClassPath(
+        logConfDir.getAbsolutePath() +
+        File.pathSeparator +
+        extraClassPath.mkString(File.pathSeparator))
+      .asScala
+      .mkString(File.pathSeparator)
+
+    props.put("spark.driver.extraClassPath", testClasspath)
+    props.put("spark.executor.extraClassPath", testClasspath)
+
+    // SPARK-4267: make sure java options are propagated correctly.
+    props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"")
+    props.setProperty("spark.executor.extraJavaOptions", "-Dfoo=\"one two three\"")
+
+    yarnCluster.getConfig().asScala.foreach { e =>
+      props.setProperty("spark.hadoop." + e.getKey(), e.getValue())
+    }
+    sys.props.foreach { case (k, v) =>
+      if (k.startsWith("spark.")) {
+        props.setProperty(k, v)
+      }
+    }
+    extraConf.foreach { case (k, v) => props.setProperty(k, v) }
+
+    val propsFile = File.createTempFile("spark", ".properties", tempDir)
+    val writer = new OutputStreamWriter(new FileOutputStream(propsFile), StandardCharsets.UTF_8)
+    props.store(writer, "Spark properties.")
+    writer.close()
+    propsFile.getAbsolutePath()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
new file mode 100644
index 0000000..b696e08
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
@@ -0,0 +1,204 @@
+/*
+ * 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 java.net.URI
+
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.Map
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.FileStatus
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.yarn.api.records.LocalResource
+import org.apache.hadoop.yarn.api.records.LocalResourceType
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
+import org.apache.hadoop.yarn.util.ConverterUtils
+import org.mockito.Mockito.when
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.yarn.config._
+
+class ClientDistributedCacheManagerSuite extends SparkFunSuite with MockitoSugar {
+
+  class MockClientDistributedCacheManager extends ClientDistributedCacheManager {
+    override def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]):
+        LocalResourceVisibility = {
+      LocalResourceVisibility.PRIVATE
+    }
+  }
+
+  test("test getFileStatus empty") {
+    val distMgr = new ClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val uri = new URI("/tmp/testing")
+    when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus())
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    val stat = distMgr.getFileStatus(fs, uri, statCache)
+    assert(stat.getPath() === null)
+  }
+
+  test("test getFileStatus cached") {
+    val distMgr = new ClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val uri = new URI("/tmp/testing")
+    val realFileStatus = new FileStatus(10, false, 1, 1024, 10, 10, null, "testOwner",
+      null, new Path("/tmp/testing"))
+    when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus())
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus](uri -> realFileStatus)
+    val stat = distMgr.getFileStatus(fs, uri, statCache)
+    assert(stat.getPath().toString() === "/tmp/testing")
+  }
+
+  test("test addResource") {
+    val distMgr = new MockClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val conf = new Configuration()
+    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
+    val localResources = HashMap[String, LocalResource]()
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    when(fs.getFileStatus(destPath)).thenReturn(new FileStatus())
+
+    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, "link",
+      statCache, false)
+    val resource = localResources("link")
+    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
+    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
+    assert(resource.getTimestamp() === 0)
+    assert(resource.getSize() === 0)
+    assert(resource.getType() === LocalResourceType.FILE)
+
+    val sparkConf = new SparkConf(false)
+    distMgr.updateConfiguration(sparkConf)
+    assert(sparkConf.get(CACHED_FILES) === Seq("file:/foo.invalid.com:8080/tmp/testing#link"))
+    assert(sparkConf.get(CACHED_FILES_TIMESTAMPS) === Seq(0L))
+    assert(sparkConf.get(CACHED_FILES_SIZES) === Seq(0L))
+    assert(sparkConf.get(CACHED_FILES_VISIBILITIES) === Seq(LocalResourceVisibility.PRIVATE.name()))
+    assert(sparkConf.get(CACHED_FILES_TYPES) === Seq(LocalResourceType.FILE.name()))
+
+    // add another one and verify both there and order correct
+    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner",
+      null, new Path("/tmp/testing2"))
+    val destPath2 = new Path("file:///foo.invalid.com:8080/tmp/testing2")
+    when(fs.getFileStatus(destPath2)).thenReturn(realFileStatus)
+    distMgr.addResource(fs, conf, destPath2, localResources, LocalResourceType.FILE, "link2",
+      statCache, false)
+    val resource2 = localResources("link2")
+    assert(resource2.getVisibility() === LocalResourceVisibility.PRIVATE)
+    assert(ConverterUtils.getPathFromYarnURL(resource2.getResource()) === destPath2)
+    assert(resource2.getTimestamp() === 10)
+    assert(resource2.getSize() === 20)
+    assert(resource2.getType() === LocalResourceType.FILE)
+
+    val sparkConf2 = new SparkConf(false)
+    distMgr.updateConfiguration(sparkConf2)
+
+    val files = sparkConf2.get(CACHED_FILES)
+    val sizes = sparkConf2.get(CACHED_FILES_SIZES)
+    val timestamps = sparkConf2.get(CACHED_FILES_TIMESTAMPS)
+    val visibilities = sparkConf2.get(CACHED_FILES_VISIBILITIES)
+
+    assert(files(0) === "file:/foo.invalid.com:8080/tmp/testing#link")
+    assert(timestamps(0)  === 0)
+    assert(sizes(0)  === 0)
+    assert(visibilities(0) === LocalResourceVisibility.PRIVATE.name())
+
+    assert(files(1) === "file:/foo.invalid.com:8080/tmp/testing2#link2")
+    assert(timestamps(1)  === 10)
+    assert(sizes(1)  === 20)
+    assert(visibilities(1) === LocalResourceVisibility.PRIVATE.name())
+  }
+
+  test("test addResource link null") {
+    val distMgr = new MockClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val conf = new Configuration()
+    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
+    val localResources = HashMap[String, LocalResource]()
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    when(fs.getFileStatus(destPath)).thenReturn(new FileStatus())
+
+    intercept[Exception] {
+      distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, null,
+        statCache, false)
+    }
+    assert(localResources.get("link") === None)
+    assert(localResources.size === 0)
+  }
+
+  test("test addResource appmaster only") {
+    val distMgr = new MockClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val conf = new Configuration()
+    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
+    val localResources = HashMap[String, LocalResource]()
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner",
+      null, new Path("/tmp/testing"))
+    when(fs.getFileStatus(destPath)).thenReturn(realFileStatus)
+
+    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link",
+      statCache, true)
+    val resource = localResources("link")
+    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
+    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
+    assert(resource.getTimestamp() === 10)
+    assert(resource.getSize() === 20)
+    assert(resource.getType() === LocalResourceType.ARCHIVE)
+
+    val sparkConf = new SparkConf(false)
+    distMgr.updateConfiguration(sparkConf)
+    assert(sparkConf.get(CACHED_FILES) === Nil)
+    assert(sparkConf.get(CACHED_FILES_TIMESTAMPS) === Nil)
+    assert(sparkConf.get(CACHED_FILES_SIZES) === Nil)
+    assert(sparkConf.get(CACHED_FILES_VISIBILITIES) === Nil)
+    assert(sparkConf.get(CACHED_FILES_TYPES) === Nil)
+  }
+
+  test("test addResource archive") {
+    val distMgr = new MockClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val conf = new Configuration()
+    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
+    val localResources = HashMap[String, LocalResource]()
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner",
+      null, new Path("/tmp/testing"))
+    when(fs.getFileStatus(destPath)).thenReturn(realFileStatus)
+
+    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link",
+      statCache, false)
+    val resource = localResources("link")
+    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
+    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
+    assert(resource.getTimestamp() === 10)
+    assert(resource.getSize() === 20)
+    assert(resource.getType() === LocalResourceType.ARCHIVE)
+
+    val sparkConf = new SparkConf(false)
+    distMgr.updateConfiguration(sparkConf)
+    assert(sparkConf.get(CACHED_FILES) === Seq("file:/foo.invalid.com:8080/tmp/testing#link"))
+    assert(sparkConf.get(CACHED_FILES_SIZES) === Seq(20L))
+    assert(sparkConf.get(CACHED_FILES_TIMESTAMPS) === Seq(10L))
+    assert(sparkConf.get(CACHED_FILES_VISIBILITIES) === Seq(LocalResourceVisibility.PRIVATE.name()))
+    assert(sparkConf.get(CACHED_FILES_TYPES) === Seq(LocalResourceType.ARCHIVE.name()))
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
new file mode 100644
index 0000000..7deaf0a
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
@@ -0,0 +1,462 @@
+/*
+ * 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 java.io.{File, FileInputStream, FileOutputStream}
+import java.net.URI
+import java.util.Properties
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{HashMap => MutableHashMap}
+import scala.reflect.ClassTag
+import scala.util.Try
+
+import org.apache.commons.lang3.SerializationUtils
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapreduce.MRJobConfig
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.YarnClientApplication
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.Records
+import org.mockito.Matchers.{eq => meq, _}
+import org.mockito.Mockito._
+import org.scalatest.{BeforeAndAfterAll, Matchers}
+
+import org.apache.spark.{SparkConf, SparkFunSuite, TestUtils}
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.util.{ResetSystemProperties, SparkConfWithEnv, Utils}
+
+class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll
+  with ResetSystemProperties {
+
+  import Client._
+
+  var oldSystemProperties: Properties = null
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    oldSystemProperties = SerializationUtils.clone(System.getProperties)
+    System.setProperty("SPARK_YARN_MODE", "true")
+  }
+
+  override def afterAll(): Unit = {
+    try {
+      System.setProperties(oldSystemProperties)
+      oldSystemProperties = null
+    } finally {
+      super.afterAll()
+    }
+  }
+
+  test("default Yarn application classpath") {
+    getDefaultYarnApplicationClasspath should be(Some(Fixtures.knownDefYarnAppCP))
+  }
+
+  test("default MR application classpath") {
+    getDefaultMRApplicationClasspath should be(Some(Fixtures.knownDefMRAppCP))
+  }
+
+  test("resultant classpath for an application that defines a classpath for YARN") {
+    withAppConf(Fixtures.mapYARNAppConf) { conf =>
+      val env = newEnv
+      populateHadoopClasspath(conf, env)
+      classpath(env) should be(
+        flatten(Fixtures.knownYARNAppCP, getDefaultMRApplicationClasspath))
+    }
+  }
+
+  test("resultant classpath for an application that defines a classpath for MR") {
+    withAppConf(Fixtures.mapMRAppConf) { conf =>
+      val env = newEnv
+      populateHadoopClasspath(conf, env)
+      classpath(env) should be(
+        flatten(getDefaultYarnApplicationClasspath, Fixtures.knownMRAppCP))
+    }
+  }
+
+  test("resultant classpath for an application that defines both classpaths, YARN and MR") {
+    withAppConf(Fixtures.mapAppConf) { conf =>
+      val env = newEnv
+      populateHadoopClasspath(conf, env)
+      classpath(env) should be(flatten(Fixtures.knownYARNAppCP, Fixtures.knownMRAppCP))
+    }
+  }
+
+  private val SPARK = "local:/sparkJar"
+  private val USER = "local:/userJar"
+  private val ADDED = "local:/addJar1,local:/addJar2,/addJar3"
+
+  private val PWD =
+    if (classOf[Environment].getMethods().exists(_.getName == "$$")) {
+      "{{PWD}}"
+    } else if (Utils.isWindows) {
+      "%PWD%"
+    } else {
+      Environment.PWD.$()
+    }
+
+  test("Local jar URIs") {
+    val conf = new Configuration()
+    val sparkConf = new SparkConf()
+      .set(SPARK_JARS, Seq(SPARK))
+      .set(USER_CLASS_PATH_FIRST, true)
+      .set("spark.yarn.dist.jars", ADDED)
+    val env = new MutableHashMap[String, String]()
+    val args = new ClientArguments(Array("--jar", USER))
+
+    populateClasspath(args, conf, sparkConf, env)
+
+    val cp = env("CLASSPATH").split(":|;|<CPS>")
+    s"$SPARK,$USER,$ADDED".split(",").foreach({ entry =>
+      val uri = new URI(entry)
+      if (LOCAL_SCHEME.equals(uri.getScheme())) {
+        cp should contain (uri.getPath())
+      } else {
+        cp should not contain (uri.getPath())
+      }
+    })
+    cp should contain(PWD)
+    cp should contain (s"$PWD${Path.SEPARATOR}${LOCALIZED_CONF_DIR}")
+    cp should not contain (APP_JAR)
+  }
+
+  test("Jar path propagation through SparkConf") {
+    val conf = new Configuration()
+    val sparkConf = new SparkConf()
+      .set(SPARK_JARS, Seq(SPARK))
+      .set("spark.yarn.dist.jars", ADDED)
+    val client = createClient(sparkConf, args = Array("--jar", USER))
+    doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]),
+      any(classOf[Path]), anyShort(), anyBoolean(), any())
+
+    val tempDir = Utils.createTempDir()
+    try {
+      // Because we mocked "copyFileToRemote" above to avoid having to create fake local files,
+      // we need to create a fake config archive in the temp dir to avoid having
+      // prepareLocalResources throw an exception.
+      new FileOutputStream(new File(tempDir, LOCALIZED_CONF_ARCHIVE)).close()
+
+      client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil)
+      sparkConf.get(APP_JAR) should be (Some(USER))
+
+      // The non-local path should be propagated by name only, since it will end up in the app's
+      // staging dir.
+      val expected = ADDED.split(",")
+        .map(p => {
+          val uri = new URI(p)
+          if (LOCAL_SCHEME == uri.getScheme()) {
+            p
+          } else {
+            Option(uri.getFragment()).getOrElse(new File(p).getName())
+          }
+        })
+        .mkString(",")
+
+      sparkConf.get(SECONDARY_JARS) should be (Some(expected.split(",").toSeq))
+    } finally {
+      Utils.deleteRecursively(tempDir)
+    }
+  }
+
+  test("Cluster path translation") {
+    val conf = new Configuration()
+    val sparkConf = new SparkConf()
+      .set(SPARK_JARS, Seq("local:/localPath/spark.jar"))
+      .set(GATEWAY_ROOT_PATH, "/localPath")
+      .set(REPLACEMENT_ROOT_PATH, "/remotePath")
+
+    getClusterPath(sparkConf, "/localPath") should be ("/remotePath")
+    getClusterPath(sparkConf, "/localPath/1:/localPath/2") should be (
+      "/remotePath/1:/remotePath/2")
+
+    val env = new MutableHashMap[String, String]()
+    populateClasspath(null, conf, sparkConf, env, extraClassPath = Some("/localPath/my1.jar"))
+    val cp = classpath(env)
+    cp should contain ("/remotePath/spark.jar")
+    cp should contain ("/remotePath/my1.jar")
+  }
+
+  test("configuration and args propagate through createApplicationSubmissionContext") {
+    val conf = new Configuration()
+    // When parsing tags, duplicates and leading/trailing whitespace should be removed.
+    // Spaces between non-comma strings should be preserved as single tags. Empty strings may or
+    // may not be removed depending on the version of Hadoop being used.
+    val sparkConf = new SparkConf()
+      .set(APPLICATION_TAGS.key, ",tag1, dup,tag2 , ,multi word , dup")
+      .set(MAX_APP_ATTEMPTS, 42)
+      .set("spark.app.name", "foo-test-app")
+      .set(QUEUE_NAME, "staging-queue")
+    val args = new ClientArguments(Array())
+
+    val appContext = Records.newRecord(classOf[ApplicationSubmissionContext])
+    val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse])
+    val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext])
+
+    val client = new Client(args, conf, sparkConf)
+    client.createApplicationSubmissionContext(
+      new YarnClientApplication(getNewApplicationResponse, appContext),
+      containerLaunchContext)
+
+    appContext.getApplicationName should be ("foo-test-app")
+    appContext.getQueue should be ("staging-queue")
+    appContext.getAMContainerSpec should be (containerLaunchContext)
+    appContext.getApplicationType should be ("SPARK")
+    appContext.getClass.getMethods.filter(_.getName.equals("getApplicationTags")).foreach{ method =>
+      val tags = method.invoke(appContext).asInstanceOf[java.util.Set[String]]
+      tags should contain allOf ("tag1", "dup", "tag2", "multi word")
+      tags.asScala.count(_.nonEmpty) should be (4)
+    }
+    appContext.getMaxAppAttempts should be (42)
+  }
+
+  test("spark.yarn.jars with multiple paths and globs") {
+    val libs = Utils.createTempDir()
+    val single = Utils.createTempDir()
+    val jar1 = TestUtils.createJarWithFiles(Map(), libs)
+    val jar2 = TestUtils.createJarWithFiles(Map(), libs)
+    val jar3 = TestUtils.createJarWithFiles(Map(), single)
+    val jar4 = TestUtils.createJarWithFiles(Map(), single)
+
+    val jarsConf = Seq(
+      s"${libs.getAbsolutePath()}/*",
+      jar3.getPath(),
+      s"local:${jar4.getPath()}",
+      s"local:${single.getAbsolutePath()}/*")
+
+    val sparkConf = new SparkConf().set(SPARK_JARS, jarsConf)
+    val client = createClient(sparkConf)
+
+    val tempDir = Utils.createTempDir()
+    client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil)
+
+    assert(sparkConf.get(SPARK_JARS) ===
+      Some(Seq(s"local:${jar4.getPath()}", s"local:${single.getAbsolutePath()}/*")))
+
+    verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar1.toURI())), anyShort(),
+      anyBoolean(), any())
+    verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar2.toURI())), anyShort(),
+      anyBoolean(), any())
+    verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar3.toURI())), anyShort(),
+      anyBoolean(), any())
+
+    val cp = classpath(client)
+    cp should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*"))
+    cp should not contain (jar3.getPath())
+    cp should contain (jar4.getPath())
+    cp should contain (buildPath(single.getAbsolutePath(), "*"))
+  }
+
+  test("distribute jars archive") {
+    val temp = Utils.createTempDir()
+    val archive = TestUtils.createJarWithFiles(Map(), temp)
+
+    val sparkConf = new SparkConf().set(SPARK_ARCHIVE, archive.getPath())
+    val client = createClient(sparkConf)
+    client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil)
+
+    verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(archive.toURI())), anyShort(),
+      anyBoolean(), any())
+    classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*"))
+
+    sparkConf.set(SPARK_ARCHIVE, LOCAL_SCHEME + ":" + archive.getPath())
+    intercept[IllegalArgumentException] {
+      client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil)
+    }
+  }
+
+  test("distribute archive multiple times") {
+    val libs = Utils.createTempDir()
+    // Create jars dir and RELEASE file to avoid IllegalStateException.
+    val jarsDir = new File(libs, "jars")
+    assert(jarsDir.mkdir())
+    new FileOutputStream(new File(libs, "RELEASE")).close()
+
+    val userLib1 = Utils.createTempDir()
+    val testJar = TestUtils.createJarWithFiles(Map(), userLib1)
+
+    // Case 1:  FILES_TO_DISTRIBUTE and ARCHIVES_TO_DISTRIBUTE can't have duplicate files
+    val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
+      .set(FILES_TO_DISTRIBUTE, Seq(testJar.getPath))
+      .set(ARCHIVES_TO_DISTRIBUTE, Seq(testJar.getPath))
+
+    val client = createClient(sparkConf)
+    val tempDir = Utils.createTempDir()
+    intercept[IllegalArgumentException] {
+      client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil)
+    }
+
+    // Case 2: FILES_TO_DISTRIBUTE can't have duplicate files.
+    val sparkConfFiles = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
+      .set(FILES_TO_DISTRIBUTE, Seq(testJar.getPath, testJar.getPath))
+
+    val clientFiles = createClient(sparkConfFiles)
+    val tempDirForFiles = Utils.createTempDir()
+    intercept[IllegalArgumentException] {
+      clientFiles.prepareLocalResources(new Path(tempDirForFiles.getAbsolutePath()), Nil)
+    }
+
+    // Case 3: ARCHIVES_TO_DISTRIBUTE can't have duplicate files.
+    val sparkConfArchives = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
+      .set(ARCHIVES_TO_DISTRIBUTE, Seq(testJar.getPath, testJar.getPath))
+
+    val clientArchives = createClient(sparkConfArchives)
+    val tempDirForArchives = Utils.createTempDir()
+    intercept[IllegalArgumentException] {
+      clientArchives.prepareLocalResources(new Path(tempDirForArchives.getAbsolutePath()), Nil)
+    }
+
+    // Case 4: FILES_TO_DISTRIBUTE can have unique file.
+    val sparkConfFilesUniq = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
+      .set(FILES_TO_DISTRIBUTE, Seq(testJar.getPath))
+
+    val clientFilesUniq = createClient(sparkConfFilesUniq)
+    val tempDirForFilesUniq = Utils.createTempDir()
+    clientFilesUniq.prepareLocalResources(new Path(tempDirForFilesUniq.getAbsolutePath()), Nil)
+
+    // Case 5: ARCHIVES_TO_DISTRIBUTE can have unique file.
+    val sparkConfArchivesUniq = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
+      .set(ARCHIVES_TO_DISTRIBUTE, Seq(testJar.getPath))
+
+    val clientArchivesUniq = createClient(sparkConfArchivesUniq)
+    val tempDirArchivesUniq = Utils.createTempDir()
+    clientArchivesUniq.prepareLocalResources(new Path(tempDirArchivesUniq.getAbsolutePath()), Nil)
+
+  }
+
+  test("distribute local spark jars") {
+    val temp = Utils.createTempDir()
+    val jarsDir = new File(temp, "jars")
+    assert(jarsDir.mkdir())
+    val jar = TestUtils.createJarWithFiles(Map(), jarsDir)
+    new FileOutputStream(new File(temp, "RELEASE")).close()
+
+    val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> temp.getAbsolutePath()))
+    val client = createClient(sparkConf)
+    client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil)
+    classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*"))
+  }
+
+  test("ignore same name jars") {
+    val libs = Utils.createTempDir()
+    val jarsDir = new File(libs, "jars")
+    assert(jarsDir.mkdir())
+    new FileOutputStream(new File(libs, "RELEASE")).close()
+    val userLib1 = Utils.createTempDir()
+    val userLib2 = Utils.createTempDir()
+
+    val jar1 = TestUtils.createJarWithFiles(Map(), jarsDir)
+    val jar2 = TestUtils.createJarWithFiles(Map(), userLib1)
+    // Copy jar2 to jar3 with same name
+    val jar3 = {
+      val target = new File(userLib2, new File(jar2.toURI).getName)
+      val input = new FileInputStream(jar2.getPath)
+      val output = new FileOutputStream(target)
+      Utils.copyStream(input, output, closeStreams = true)
+      target.toURI.toURL
+    }
+
+    val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
+      .set(JARS_TO_DISTRIBUTE, Seq(jar2.getPath, jar3.getPath))
+
+    val client = createClient(sparkConf)
+    val tempDir = Utils.createTempDir()
+    client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil)
+
+    // Only jar2 will be added to SECONDARY_JARS, jar3 which has the same name with jar2 will be
+    // ignored.
+    sparkConf.get(SECONDARY_JARS) should be (Some(Seq(new File(jar2.toURI).getName)))
+  }
+
+  object Fixtures {
+
+    val knownDefYarnAppCP: Seq[String] =
+      getFieldValue[Array[String], Seq[String]](classOf[YarnConfiguration],
+                                                "DEFAULT_YARN_APPLICATION_CLASSPATH",
+                                                Seq[String]())(a => a.toSeq)
+
+
+    val knownDefMRAppCP: Seq[String] =
+      getFieldValue2[String, Array[String], Seq[String]](
+        classOf[MRJobConfig],
+        "DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH",
+        Seq[String]())(a => a.split(","))(a => a.toSeq)
+
+    val knownYARNAppCP = Some(Seq("/known/yarn/path"))
+
+    val knownMRAppCP = Some(Seq("/known/mr/path"))
+
+    val mapMRAppConf =
+      Map("mapreduce.application.classpath" -> knownMRAppCP.map(_.mkString(":")).get)
+
+    val mapYARNAppConf =
+      Map(YarnConfiguration.YARN_APPLICATION_CLASSPATH -> knownYARNAppCP.map(_.mkString(":")).get)
+
+    val mapAppConf = mapYARNAppConf ++ mapMRAppConf
+  }
+
+  def withAppConf(m: Map[String, String] = Map())(testCode: (Configuration) => Any) {
+    val conf = new Configuration
+    m.foreach { case (k, v) => conf.set(k, v, "ClientSpec") }
+    testCode(conf)
+  }
+
+  def newEnv: MutableHashMap[String, String] = MutableHashMap[String, String]()
+
+  def classpath(env: MutableHashMap[String, String]): Array[String] =
+    env(Environment.CLASSPATH.name).split(":|;|<CPS>")
+
+  def flatten(a: Option[Seq[String]], b: Option[Seq[String]]): Array[String] =
+    (a ++ b).flatten.toArray
+
+  def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B = {
+    Try(clazz.getField(field))
+      .map(_.get(null).asInstanceOf[A])
+      .toOption
+      .map(mapTo)
+      .getOrElse(defaults)
+  }
+
+  def getFieldValue2[A: ClassTag, A1: ClassTag, B](
+        clazz: Class[_],
+        field: String,
+        defaults: => B)(mapTo: A => B)(mapTo1: A1 => B): B = {
+    Try(clazz.getField(field)).map(_.get(null)).map {
+      case v: A => mapTo(v)
+      case v1: A1 => mapTo1(v1)
+      case _ => defaults
+    }.toOption.getOrElse(defaults)
+  }
+
+  private def createClient(
+      sparkConf: SparkConf,
+      conf: Configuration = new Configuration(),
+      args: Array[String] = Array()): Client = {
+    val clientArgs = new ClientArguments(args)
+    spy(new Client(clientArgs, conf, sparkConf))
+  }
+
+  private def classpath(client: Client): Array[String] = {
+    val env = new MutableHashMap[String, String]()
+    populateClasspath(null, client.hadoopConf, client.sparkConf, env)
+    classpath(env)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala
new file mode 100644
index 0000000..afb4b69
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala
@@ -0,0 +1,153 @@
+/*
+ * 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.yarn.client.api.AMRMClient.ContainerRequest
+import org.scalatest.{BeforeAndAfterEach, Matchers}
+
+import org.apache.spark.SparkFunSuite
+
+class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with BeforeAndAfterEach {
+
+  private val yarnAllocatorSuite = new YarnAllocatorSuite
+  import yarnAllocatorSuite._
+
+  def createContainerRequest(nodes: Array[String]): ContainerRequest =
+    new ContainerRequest(containerResource, nodes, null, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)
+
+  override def beforeEach() {
+    yarnAllocatorSuite.beforeEach()
+  }
+
+  override def afterEach() {
+    yarnAllocatorSuite.afterEach()
+  }
+
+  test("allocate locality preferred containers with enough resource and no matched existed " +
+    "containers") {
+    // 1. All the locations of current containers cannot satisfy the new requirements
+    // 2. Current requested container number can fully satisfy the pending tasks.
+
+    val handler = createAllocator(2)
+    handler.updateResourceRequests()
+    handler.handleAllocatedContainers(Array(createContainer("host1"), createContainer("host2")))
+
+    val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
+      3, 15, Map("host3" -> 15, "host4" -> 15, "host5" -> 10),
+        handler.allocatedHostToContainersMap, Seq.empty)
+
+    assert(localities.map(_.nodes) === Array(
+      Array("host3", "host4", "host5"),
+      Array("host3", "host4", "host5"),
+      Array("host3", "host4")))
+  }
+
+  test("allocate locality preferred containers with enough resource and partially matched " +
+    "containers") {
+    // 1. Parts of current containers' locations can satisfy the new requirements
+    // 2. Current requested container number can fully satisfy the pending tasks.
+
+    val handler = createAllocator(3)
+    handler.updateResourceRequests()
+    handler.handleAllocatedContainers(Array(
+      createContainer("host1"),
+      createContainer("host1"),
+      createContainer("host2")
+    ))
+
+    val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
+      3, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
+        handler.allocatedHostToContainersMap, Seq.empty)
+
+    assert(localities.map(_.nodes) ===
+      Array(null, Array("host2", "host3"), Array("host2", "host3")))
+  }
+
+  test("allocate locality preferred containers with limited resource and partially matched " +
+    "containers") {
+    // 1. Parts of current containers' locations can satisfy the new requirements
+    // 2. Current requested container number cannot fully satisfy the pending tasks.
+
+    val handler = createAllocator(3)
+    handler.updateResourceRequests()
+    handler.handleAllocatedContainers(Array(
+      createContainer("host1"),
+      createContainer("host1"),
+      createContainer("host2")
+    ))
+
+    val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
+      1, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
+        handler.allocatedHostToContainersMap, Seq.empty)
+
+    assert(localities.map(_.nodes) === Array(Array("host2", "host3")))
+  }
+
+  test("allocate locality preferred containers with fully matched containers") {
+    // Current containers' locations can fully satisfy the new requirements
+
+    val handler = createAllocator(5)
+    handler.updateResourceRequests()
+    handler.handleAllocatedContainers(Array(
+      createContainer("host1"),
+      createContainer("host1"),
+      createContainer("host2"),
+      createContainer("host2"),
+      createContainer("host3")
+    ))
+
+    val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
+      3, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
+        handler.allocatedHostToContainersMap, Seq.empty)
+
+    assert(localities.map(_.nodes) === Array(null, null, null))
+  }
+
+  test("allocate containers with no locality preference") {
+    // Request new container without locality preference
+
+    val handler = createAllocator(2)
+    handler.updateResourceRequests()
+    handler.handleAllocatedContainers(Array(createContainer("host1"), createContainer("host2")))
+
+    val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
+      1, 0, Map.empty, handler.allocatedHostToContainersMap, Seq.empty)
+
+    assert(localities.map(_.nodes) === Array(null))
+  }
+
+  test("allocate locality preferred containers by considering the localities of pending requests") {
+    val handler = createAllocator(3)
+    handler.updateResourceRequests()
+    handler.handleAllocatedContainers(Array(
+      createContainer("host1"),
+      createContainer("host1"),
+      createContainer("host2")
+    ))
+
+    val pendingAllocationRequests = Seq(
+      createContainerRequest(Array("host2", "host3")),
+      createContainerRequest(Array("host1", "host4")))
+
+    val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
+      1, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
+        handler.allocatedHostToContainersMap, pendingAllocationRequests)
+
+    assert(localities.map(_.nodes) === Array(Array("host3")))
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..994dc75
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala
@@ -0,0 +1,344 @@
+/*
+ * 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 java.util.{Arrays, List => JList}
+
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic
+import org.apache.hadoop.net.DNSToSwitchMapping
+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.mockito.Mockito._
+import org.scalatest.{BeforeAndAfterEach, Matchers}
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.deploy.yarn.YarnAllocator._
+import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._
+import org.apache.spark.rpc.RpcEndpointRef
+import org.apache.spark.scheduler.SplitInfo
+import org.apache.spark.util.ManualClock
+
+class MockResolver extends DNSToSwitchMapping {
+
+  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 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")
+  sparkConf.set(SPARK_JARS, Seq("notarealjar.jar"))
+  sparkConf.set("spark.yarn.launchContainers", "false")
+
+  val appAttemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 0), 0)
+
+  // Resource returned by YARN.  YARN can give larger containers than requested, so give 6 cores
+  // instead of the 5 requested and 3 GB instead of the 2 requested.
+  val containerResource = Resource.newInstance(3072, 6)
+
+  var rmClient: AMRMClient[ContainerRequest] = _
+
+  var containerNum = 0
+
+  override def beforeEach() {
+    super.beforeEach()
+    rmClient = AMRMClient.createAMRMClient()
+    rmClient.init(conf)
+    rmClient.start()
+  }
+
+  override def afterEach() {
+    try {
+      rmClient.stop()
+    } finally {
+      super.afterEach()
+    }
+  }
+
+  class MockSplitInfo(host: String) extends SplitInfo(null, host, null, 1, null) {
+    override def hashCode(): Int = 0
+    override def equals(other: Any): Boolean = false
+  }
+
+  def createAllocator(maxExecutors: Int = 5): YarnAllocator = {
+    val args = Array(
+      "--jar", "somejar.jar",
+      "--class", "SomeClass")
+    val sparkConfClone = sparkConf.clone()
+    sparkConfClone
+      .set("spark.executor.instances", maxExecutors.toString)
+      .set("spark.executor.cores", "5")
+      .set("spark.executor.memory", "2048")
+    new YarnAllocator(
+      "not used",
+      mock(classOf[RpcEndpointRef]),
+      conf,
+      sparkConfClone,
+      rmClient,
+      appAttemptId,
+      new SecurityManager(sparkConf),
+      Map())
+  }
+
+  def createContainer(host: String): Container = {
+    // When YARN 2.6+ is required, avoid deprecation by using version with long second arg
+    val containerId = ContainerId.newInstance(appAttemptId, containerNum)
+    containerNum += 1
+    val nodeId = NodeId.newInstance(host, 1000)
+    Container.newInstance(containerId, nodeId, "", containerResource, RM_REQUEST_PRIORITY, null)
+  }
+
+  test("single container allocated") {
+    // request a single container and receive it
+    val handler = createAllocator(1)
+    handler.updateResourceRequests()
+    handler.getNumExecutorsRunning should be (0)
+    handler.getPendingAllocate.size should be (1)
+
+    val container = createContainer("host1")
+    handler.handleAllocatedContainers(Array(container))
+
+    handler.getNumExecutorsRunning should be (1)
+    handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1")
+    handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId)
+
+    val size = rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size
+    size should be (0)
+  }
+
+  test("container should not be created if requested number if met") {
+    // request a single container and receive it
+    val handler = createAllocator(1)
+    handler.updateResourceRequests()
+    handler.getNumExecutorsRunning should be (0)
+    handler.getPendingAllocate.size should be (1)
+
+    val container = createContainer("host1")
+    handler.handleAllocatedContainers(Array(container))
+
+    handler.getNumExecutorsRunning should be (1)
+    handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1")
+    handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId)
+
+    val container2 = createContainer("host2")
+    handler.handleAllocatedContainers(Array(container2))
+    handler.getNumExecutorsRunning should be (1)
+  }
+
+  test("some containers allocated") {
+    // request a few containers and receive some of them
+    val handler = createAllocator(4)
+    handler.updateResourceRequests()
+    handler.getNumExecutorsRunning should be (0)
+    handler.getPendingAllocate.size should be (4)
+
+    val container1 = createContainer("host1")
+    val container2 = createContainer("host1")
+    val container3 = createContainer("host2")
+    handler.handleAllocatedContainers(Array(container1, container2, container3))
+
+    handler.getNumExecutorsRunning should be (3)
+    handler.allocatedContainerToHostMap.get(container1.getId).get should be ("host1")
+    handler.allocatedContainerToHostMap.get(container2.getId).get should be ("host1")
+    handler.allocatedContainerToHostMap.get(container3.getId).get should be ("host2")
+    handler.allocatedHostToContainersMap.get("host1").get should contain (container1.getId)
+    handler.allocatedHostToContainersMap.get("host1").get should contain (container2.getId)
+    handler.allocatedHostToContainersMap.get("host2").get should contain (container3.getId)
+  }
+
+  test("receive more containers than requested") {
+    val handler = createAllocator(2)
+    handler.updateResourceRequests()
+    handler.getNumExecutorsRunning should be (0)
+    handler.getPendingAllocate.size should be (2)
+
+    val container1 = createContainer("host1")
+    val container2 = createContainer("host2")
+    val container3 = createContainer("host4")
+    handler.handleAllocatedContainers(Array(container1, container2, container3))
+
+    handler.getNumExecutorsRunning should be (2)
+    handler.allocatedContainerToHostMap.get(container1.getId).get should be ("host1")
+    handler.allocatedContainerToHostMap.get(container2.getId).get should be ("host2")
+    handler.allocatedContainerToHostMap.contains(container3.getId) should be (false)
+    handler.allocatedHostToContainersMap.get("host1").get should contain (container1.getId)
+    handler.allocatedHostToContainersMap.get("host2").get should contain (container2.getId)
+    handler.allocatedHostToContainersMap.contains("host4") should be (false)
+  }
+
+  test("decrease total requested executors") {
+    val handler = createAllocator(4)
+    handler.updateResourceRequests()
+    handler.getNumExecutorsRunning should be (0)
+    handler.getPendingAllocate.size should be (4)
+
+    handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty)
+    handler.updateResourceRequests()
+    handler.getPendingAllocate.size should be (3)
+
+    val container = createContainer("host1")
+    handler.handleAllocatedContainers(Array(container))
+
+    handler.getNumExecutorsRunning should be (1)
+    handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1")
+    handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId)
+
+    handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map.empty)
+    handler.updateResourceRequests()
+    handler.getPendingAllocate.size should be (1)
+  }
+
+  test("decrease total requested executors to less than currently running") {
+    val handler = createAllocator(4)
+    handler.updateResourceRequests()
+    handler.getNumExecutorsRunning should be (0)
+    handler.getPendingAllocate.size should be (4)
+
+    handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty)
+    handler.updateResourceRequests()
+    handler.getPendingAllocate.size should be (3)
+
+    val container1 = createContainer("host1")
+    val container2 = createContainer("host2")
+    handler.handleAllocatedContainers(Array(container1, container2))
+
+    handler.getNumExecutorsRunning should be (2)
+
+    handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty)
+    handler.updateResourceRequests()
+    handler.getPendingAllocate.size should be (0)
+    handler.getNumExecutorsRunning should be (2)
+  }
+
+  test("kill executors") {
+    val handler = createAllocator(4)
+    handler.updateResourceRequests()
+    handler.getNumExecutorsRunning should be (0)
+    handler.getPendingAllocate.size should be (4)
+
+    val container1 = createContainer("host1")
+    val container2 = createContainer("host2")
+    handler.handleAllocatedContainers(Array(container1, container2))
+
+    handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty)
+    handler.executorIdToContainer.keys.foreach { id => handler.killExecutor(id ) }
+
+    val statuses = Seq(container1, container2).map { c =>
+      ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Finished", 0)
+    }
+    handler.updateResourceRequests()
+    handler.processCompletedContainers(statuses.toSeq)
+    handler.getNumExecutorsRunning should be (0)
+    handler.getPendingAllocate.size should be (1)
+  }
+
+  test("lost executor removed from backend") {
+    val handler = createAllocator(4)
+    handler.updateResourceRequests()
+    handler.getNumExecutorsRunning should be (0)
+    handler.getPendingAllocate.size should be (4)
+
+    val container1 = createContainer("host1")
+    val container2 = createContainer("host2")
+    handler.handleAllocatedContainers(Array(container1, container2))
+
+    handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map())
+
+    val statuses = Seq(container1, container2).map { c =>
+      ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Failed", -1)
+    }
+    handler.updateResourceRequests()
+    handler.processCompletedContainers(statuses.toSeq)
+    handler.updateResourceRequests()
+    handler.getNumExecutorsRunning should be (0)
+    handler.getPendingAllocate.size should be (2)
+    handler.getNumExecutorsFailed should be (2)
+    handler.getNumUnexpectedContainerRelease should be (2)
+  }
+
+  test("memory exceeded diagnostic regexes") {
+    val diagnostics =
+      "Container [pid=12465,containerID=container_1412887393566_0003_01_000002] is running " +
+        "beyond physical memory limits. Current usage: 2.1 MB of 2 GB physical memory used; " +
+        "5.8 GB of 4.2 GB virtual memory used. Killing container."
+    val vmemMsg = memLimitExceededLogMessage(diagnostics, VMEM_EXCEEDED_PATTERN)
+    val pmemMsg = memLimitExceededLogMessage(diagnostics, PMEM_EXCEEDED_PATTERN)
+    assert(vmemMsg.contains("5.8 GB of 4.2 GB virtual memory used."))
+    assert(pmemMsg.contains("2.1 MB of 2 GB physical memory used."))
+  }
+
+  test("window based failure executor counting") {
+    sparkConf.set("spark.yarn.executor.failuresValidityInterval", "100s")
+    val handler = createAllocator(4)
+    val clock = new ManualClock(0L)
+    handler.setClock(clock)
+
+    handler.updateResourceRequests()
+    handler.getNumExecutorsRunning should be (0)
+    handler.getPendingAllocate.size should be (4)
+
+    val containers = Seq(
+      createContainer("host1"),
+      createContainer("host2"),
+      createContainer("host3"),
+      createContainer("host4")
+    )
+    handler.handleAllocatedContainers(containers)
+
+    val failedStatuses = containers.map { c =>
+      ContainerStatus.newInstance(c.getId, ContainerState.COMPLETE, "Failed", -1)
+    }
+
+    handler.getNumExecutorsFailed should be (0)
+
+    clock.advance(100 * 1000L)
+    handler.processCompletedContainers(failedStatuses.slice(0, 1))
+    handler.getNumExecutorsFailed should be (1)
+
+    clock.advance(101 * 1000L)
+    handler.getNumExecutorsFailed should be (0)
+
+    handler.processCompletedContainers(failedStatuses.slice(1, 3))
+    handler.getNumExecutorsFailed should be (2)
+
+    clock.advance(50 * 1000L)
+    handler.processCompletedContainers(failedStatuses.slice(3, 4))
+    handler.getNumExecutorsFailed should be (3)
+
+    clock.advance(51 * 1000L)
+    handler.getNumExecutorsFailed should be (1)
+
+    clock.advance(50 * 1000L)
+    handler.getNumExecutorsFailed should be (0)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
new file mode 100644
index 0000000..99fb58a
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -0,0 +1,493 @@
+/*
+ * 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 java.io.File
+import java.net.URL
+import java.nio.charset.StandardCharsets
+import java.util.{HashMap => JHashMap}
+
+import scala.collection.mutable
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import com.google.common.io.{ByteStreams, Files}
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.scalatest.Matchers
+import org.scalatest.concurrent.Eventually._
+
+import org.apache.spark._
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.launcher._
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart,
+  SparkListenerExecutorAdded}
+import org.apache.spark.scheduler.cluster.ExecutorInfo
+import org.apache.spark.tags.ExtendedYarnTest
+import org.apache.spark.util.Utils
+
+/**
+ * Integration tests for YARN; these tests use a mini Yarn cluster to run Spark-on-YARN
+ * applications, and require the Spark assembly to be built before they can be successfully
+ * run.
+ */
+@ExtendedYarnTest
+class YarnClusterSuite extends BaseYarnClusterSuite {
+
+  override def newYarnConfig(): YarnConfiguration = new YarnConfiguration()
+
+  private val TEST_PYFILE = """
+    |import mod1, mod2
+    |import sys
+    |from operator import add
+    |
+    |from pyspark import SparkConf , SparkContext
+    |if __name__ == "__main__":
+    |    if len(sys.argv) != 2:
+    |        print >> sys.stderr, "Usage: test.py [result file]"
+    |        exit(-1)
+    |    sc = SparkContext(conf=SparkConf())
+    |    status = open(sys.argv[1],'w')
+    |    result = "failure"
+    |    rdd = sc.parallelize(range(10)).map(lambda x: x * mod1.func() * mod2.func())
+    |    cnt = rdd.count()
+    |    if cnt == 10:
+    |        result = "success"
+    |    status.write(result)
+    |    status.close()
+    |    sc.stop()
+    """.stripMargin
+
+  private val TEST_PYMODULE = """
+    |def func():
+    |    return 42
+    """.stripMargin
+
+  test("run Spark in yarn-client mode") {
+    testBasicYarnApp(true)
+  }
+
+  test("run Spark in yarn-cluster mode") {
+    testBasicYarnApp(false)
+  }
+
+  test("run Spark in yarn-client mode with different configurations") {
+    testBasicYarnApp(true,
+      Map(
+        "spark.driver.memory" -> "512m",
+        "spark.executor.cores" -> "1",
+        "spark.executor.memory" -> "512m",
+        "spark.executor.instances" -> "2"
+      ))
+  }
+
+  test("run Spark in yarn-cluster mode with different configurations") {
+    testBasicYarnApp(false,
+      Map(
+        "spark.driver.memory" -> "512m",
+        "spark.driver.cores" -> "1",
+        "spark.executor.cores" -> "1",
+        "spark.executor.memory" -> "512m",
+        "spark.executor.instances" -> "2"
+      ))
+  }
+
+  test("run Spark in yarn-cluster mode with using SparkHadoopUtil.conf") {
+    testYarnAppUseSparkHadoopUtilConf()
+  }
+
+  test("run Spark in yarn-client mode with additional jar") {
+    testWithAddJar(true)
+  }
+
+  test("run Spark in yarn-cluster mode with additional jar") {
+    testWithAddJar(false)
+  }
+
+  test("run Spark in yarn-cluster mode unsuccessfully") {
+    // Don't provide arguments so the driver will fail.
+    val finalState = runSpark(false, mainClassName(YarnClusterDriver.getClass))
+    finalState should be (SparkAppHandle.State.FAILED)
+  }
+
+  test("run Spark in yarn-cluster mode failure after sc initialized") {
+    val finalState = runSpark(false, mainClassName(YarnClusterDriverWithFailure.getClass))
+    finalState should be (SparkAppHandle.State.FAILED)
+  }
+
+  test("run Python application in yarn-client mode") {
+    testPySpark(true)
+  }
+
+  test("run Python application in yarn-cluster mode") {
+    testPySpark(false)
+  }
+
+  test("run Python application in yarn-cluster mode using " +
+    " spark.yarn.appMasterEnv to override local envvar") {
+    testPySpark(
+      clientMode = false,
+      extraConf = Map(
+        "spark.yarn.appMasterEnv.PYSPARK_DRIVER_PYTHON"
+          -> sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python"),
+        "spark.yarn.appMasterEnv.PYSPARK_PYTHON"
+          -> sys.env.getOrElse("PYSPARK_PYTHON", "python")),
+      extraEnv = Map(
+        "PYSPARK_DRIVER_PYTHON" -> "not python",
+        "PYSPARK_PYTHON" -> "not python"))
+  }
+
+  test("user class path first in client mode") {
+    testUseClassPathFirst(true)
+  }
+
+  test("user class path first in cluster mode") {
+    testUseClassPathFirst(false)
+  }
+
+  test("monitor app using launcher library") {
+    val env = new JHashMap[String, String]()
+    env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
+
+    val propsFile = createConfFile()
+    val handle = new SparkLauncher(env)
+      .setSparkHome(sys.props("spark.test.home"))
+      .setConf("spark.ui.enabled", "false")
+      .setPropertiesFile(propsFile)
+      .setMaster("yarn")
+      .setDeployMode("client")
+      .setAppResource(SparkLauncher.NO_RESOURCE)
+      .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
+      .startApplication()
+
+    try {
+      eventually(timeout(30 seconds), interval(100 millis)) {
+        handle.getState() should be (SparkAppHandle.State.RUNNING)
+      }
+
+      handle.getAppId() should not be (null)
+      handle.getAppId() should startWith ("application_")
+      handle.stop()
+
+      eventually(timeout(30 seconds), interval(100 millis)) {
+        handle.getState() should be (SparkAppHandle.State.KILLED)
+      }
+    } finally {
+      handle.kill()
+    }
+  }
+
+  test("timeout to get SparkContext in cluster mode triggers failure") {
+    val timeout = 2000
+    val finalState = runSpark(false, mainClassName(SparkContextTimeoutApp.getClass),
+      appArgs = Seq((timeout * 4).toString),
+      extraConf = Map(AM_MAX_WAIT_TIME.key -> timeout.toString))
+    finalState should be (SparkAppHandle.State.FAILED)
+  }
+
+  private def testBasicYarnApp(clientMode: Boolean, conf: Map[String, String] = Map()): Unit = {
+    val result = File.createTempFile("result", null, tempDir)
+    val finalState = runSpark(clientMode, mainClassName(YarnClusterDriver.getClass),
+      appArgs = Seq(result.getAbsolutePath()),
+      extraConf = conf)
+    checkResult(finalState, result)
+  }
+
+  private def testYarnAppUseSparkHadoopUtilConf(): Unit = {
+    val result = File.createTempFile("result", null, tempDir)
+    val finalState = runSpark(false,
+      mainClassName(YarnClusterDriverUseSparkHadoopUtilConf.getClass),
+      appArgs = Seq("key=value", result.getAbsolutePath()),
+      extraConf = Map("spark.hadoop.key" -> "value"))
+    checkResult(finalState, result)
+  }
+
+  private def testWithAddJar(clientMode: Boolean): Unit = {
+    val originalJar = TestUtils.createJarWithFiles(Map("test.resource" -> "ORIGINAL"), tempDir)
+    val driverResult = File.createTempFile("driver", null, tempDir)
+    val executorResult = File.createTempFile("executor", null, tempDir)
+    val finalState = runSpark(clientMode, mainClassName(YarnClasspathTest.getClass),
+      appArgs = Seq(driverResult.getAbsolutePath(), executorResult.getAbsolutePath()),
+      extraClassPath = Seq(originalJar.getPath()),
+      extraJars = Seq("local:" + originalJar.getPath()))
+    checkResult(finalState, driverResult, "ORIGINAL")
+    checkResult(finalState, executorResult, "ORIGINAL")
+  }
+
+  private def testPySpark(
+      clientMode: Boolean,
+      extraConf: Map[String, String] = Map(),
+      extraEnv: Map[String, String] = Map()): Unit = {
+    val primaryPyFile = new File(tempDir, "test.py")
+    Files.write(TEST_PYFILE, primaryPyFile, StandardCharsets.UTF_8)
+
+    // When running tests, let's not assume the user has built the assembly module, which also
+    // creates the pyspark archive. Instead, let's use PYSPARK_ARCHIVES_PATH to point at the
+    // needed locations.
+    val sparkHome = sys.props("spark.test.home")
+    val pythonPath = Seq(
+        s"$sparkHome/python/lib/py4j-0.10.4-src.zip",
+        s"$sparkHome/python")
+    val extraEnvVars = Map(
+      "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator),
+      "PYTHONPATH" -> pythonPath.mkString(File.pathSeparator)) ++ extraEnv
+
+    val moduleDir =
+      if (clientMode) {
+        // In client-mode, .py files added with --py-files are not visible in the driver.
+        // This is something that the launcher library would have to handle.
+        tempDir
+      } else {
+        val subdir = new File(tempDir, "pyModules")
+        subdir.mkdir()
+        subdir
+      }
+    val pyModule = new File(moduleDir, "mod1.py")
+    Files.write(TEST_PYMODULE, pyModule, StandardCharsets.UTF_8)
+
+    val mod2Archive = TestUtils.createJarWithFiles(Map("mod2.py" -> TEST_PYMODULE), moduleDir)
+    val pyFiles = Seq(pyModule.getAbsolutePath(), mod2Archive.getPath()).mkString(",")
+    val result = File.createTempFile("result", null, tempDir)
+
+    val finalState = runSpark(clientMode, primaryPyFile.getAbsolutePath(),
+      sparkArgs = Seq("--py-files" -> pyFiles),
+      appArgs = Seq(result.getAbsolutePath()),
+      extraEnv = extraEnvVars,
+      extraConf = extraConf)
+    checkResult(finalState, result)
+  }
+
+  private def testUseClassPathFirst(clientMode: Boolean): Unit = {
+    // Create a jar file that contains a different version of "test.resource".
+    val originalJar = TestUtils.createJarWithFiles(Map("test.resource" -> "ORIGINAL"), tempDir)
+    val userJar = TestUtils.createJarWithFiles(Map("test.resource" -> "OVERRIDDEN"), tempDir)
+    val driverResult = File.createTempFile("driver", null, tempDir)
+    val executorResult = File.createTempFile("executor", null, tempDir)
+    val finalState = runSpark(clientMode, mainClassName(YarnClasspathTest.getClass),
+      appArgs = Seq(driverResult.getAbsolutePath(), executorResult.getAbsolutePath()),
+      extraClassPath = Seq(originalJar.getPath()),
+      extraJars = Seq("local:" + userJar.getPath()),
+      extraConf = Map(
+        "spark.driver.userClassPathFirst" -> "true",
+        "spark.executor.userClassPathFirst" -> "true"))
+    checkResult(finalState, driverResult, "OVERRIDDEN")
+    checkResult(finalState, executorResult, "OVERRIDDEN")
+  }
+
+}
+
+private[spark] class SaveExecutorInfo extends SparkListener {
+  val addedExecutorInfos = mutable.Map[String, ExecutorInfo]()
+  var driverLogs: Option[collection.Map[String, String]] = None
+
+  override def onExecutorAdded(executor: SparkListenerExecutorAdded) {
+    addedExecutorInfos(executor.executorId) = executor.executorInfo
+  }
+
+  override def onApplicationStart(appStart: SparkListenerApplicationStart): Unit = {
+    driverLogs = appStart.driverLogs
+  }
+}
+
+private object YarnClusterDriverWithFailure extends Logging with Matchers {
+  def main(args: Array[String]): Unit = {
+    val sc = new SparkContext(new SparkConf()
+      .set("spark.extraListeners", classOf[SaveExecutorInfo].getName)
+      .setAppName("yarn test with failure"))
+
+    throw new Exception("exception after sc initialized")
+  }
+}
+
+private object YarnClusterDriverUseSparkHadoopUtilConf extends Logging with Matchers {
+  def main(args: Array[String]): Unit = {
+    if (args.length != 2) {
+      // scalastyle:off println
+      System.err.println(
+        s"""
+        |Invalid command line: ${args.mkString(" ")}
+        |
+        |Usage: YarnClusterDriverUseSparkHadoopUtilConf [hadoopConfKey=value] [result file]
+        """.stripMargin)
+      // scalastyle:on println
+      System.exit(1)
+    }
+
+    val sc = new SparkContext(new SparkConf()
+      .set("spark.extraListeners", classOf[SaveExecutorInfo].getName)
+      .setAppName("yarn test using SparkHadoopUtil's conf"))
+
+    val kv = args(0).split("=")
+    val status = new File(args(1))
+    var result = "failure"
+    try {
+      SparkHadoopUtil.get.conf.get(kv(0)) should be (kv(1))
+      result = "success"
+    } finally {
+      Files.write(result, status, StandardCharsets.UTF_8)
+      sc.stop()
+    }
+  }
+}
+
+private object YarnClusterDriver extends Logging with Matchers {
+
+  val WAIT_TIMEOUT_MILLIS = 10000
+
+  def main(args: Array[String]): Unit = {
+    if (args.length != 1) {
+      // scalastyle:off println
+      System.err.println(
+        s"""
+        |Invalid command line: ${args.mkString(" ")}
+        |
+        |Usage: YarnClusterDriver [result file]
+        """.stripMargin)
+      // scalastyle:on println
+      System.exit(1)
+    }
+
+    val sc = new SparkContext(new SparkConf()
+      .set("spark.extraListeners", classOf[SaveExecutorInfo].getName)
+      .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns"))
+    val conf = sc.getConf
+    val status = new File(args(0))
+    var result = "failure"
+    try {
+      val data = sc.parallelize(1 to 4, 4).collect().toSet
+      sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
+      data should be (Set(1, 2, 3, 4))
+      result = "success"
+
+      // Verify that the config archive is correctly placed in the classpath of all containers.
+      val confFile = "/" + Client.SPARK_CONF_FILE
+      assert(getClass().getResource(confFile) != null)
+      val configFromExecutors = sc.parallelize(1 to 4, 4)
+        .map { _ => Option(getClass().getResource(confFile)).map(_.toString).orNull }
+        .collect()
+      assert(configFromExecutors.find(_ == null) === None)
+    } finally {
+      Files.write(result, status, StandardCharsets.UTF_8)
+      sc.stop()
+    }
+
+    // verify log urls are present
+    val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo]
+    assert(listeners.size === 1)
+    val listener = listeners(0)
+    val executorInfos = listener.addedExecutorInfos.values
+    assert(executorInfos.nonEmpty)
+    executorInfos.foreach { info =>
+      assert(info.logUrlMap.nonEmpty)
+    }
+
+    // If we are running in yarn-cluster mode, verify that driver logs links and present and are
+    // in the expected format.
+    if (conf.get("spark.submit.deployMode") == "cluster") {
+      assert(listener.driverLogs.nonEmpty)
+      val driverLogs = listener.driverLogs.get
+      assert(driverLogs.size === 2)
+      assert(driverLogs.contains("stderr"))
+      assert(driverLogs.contains("stdout"))
+      val urlStr = driverLogs("stderr")
+      // Ensure that this is a valid URL, else this will throw an exception
+      new URL(urlStr)
+      val containerId = YarnSparkHadoopUtil.get.getContainerId
+      val user = Utils.getCurrentUserName()
+      assert(urlStr.endsWith(s"/node/containerlogs/$containerId/$user/stderr?start=-4096"))
+    }
+  }
+
+}
+
+private object YarnClasspathTest extends Logging {
+  def error(m: String, ex: Throwable = null): Unit = {
+    logError(m, ex)
+    // scalastyle:off println
+    System.out.println(m)
+    if (ex != null) {
+      ex.printStackTrace(System.out)
+    }
+    // scalastyle:on println
+  }
+
+  def main(args: Array[String]): Unit = {
+    if (args.length != 2) {
+      error(
+        s"""
+        |Invalid command line: ${args.mkString(" ")}
+        |
+        |Usage: YarnClasspathTest [driver result file] [executor result file]
+        """.stripMargin)
+      // scalastyle:on println
+    }
+
+    readResource(args(0))
+    val sc = new SparkContext(new SparkConf())
+    try {
+      sc.parallelize(Seq(1)).foreach { x => readResource(args(1)) }
+    } finally {
+      sc.stop()
+    }
+  }
+
+  private def readResource(resultPath: String): Unit = {
+    var result = "failure"
+    try {
+      val ccl = Thread.currentThread().getContextClassLoader()
+      val resource = ccl.getResourceAsStream("test.resource")
+      val bytes = ByteStreams.toByteArray(resource)
+      result = new String(bytes, 0, bytes.length, StandardCharsets.UTF_8)
+    } catch {
+      case t: Throwable =>
+        error(s"loading test.resource to $resultPath", t)
+    } finally {
+      Files.write(result, new File(resultPath), StandardCharsets.UTF_8)
+    }
+  }
+
+}
+
+private object YarnLauncherTestApp {
+
+  def main(args: Array[String]): Unit = {
+    // Do not stop the application; the test will stop it using the launcher lib. Just run a task
+    // that will prevent the process from exiting.
+    val sc = new SparkContext(new SparkConf())
+    sc.parallelize(Seq(1)).foreach { i =>
+      this.synchronized {
+        wait()
+      }
+    }
+  }
+
+}
+
+/**
+ * Used to test code in the AM that detects the SparkContext instance. Expects a single argument
+ * with the duration to sleep for, in ms.
+ */
+private object SparkContextTimeoutApp {
+
+  def main(args: Array[String]): Unit = {
+    val Array(sleepTime) = args
+    Thread.sleep(java.lang.Long.parseLong(sleepTime))
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala
new file mode 100644
index 0000000..950ebd9
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala
@@ -0,0 +1,112 @@
+/*
+* 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 java.io.File
+import java.nio.charset.StandardCharsets
+
+import com.google.common.io.Files
+import org.apache.commons.io.FileUtils
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.scalatest.Matchers
+
+import org.apache.spark._
+import org.apache.spark.internal.Logging
+import org.apache.spark.network.shuffle.ShuffleTestAccessor
+import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor}
+import org.apache.spark.tags.ExtendedYarnTest
+
+/**
+ * Integration test for the external shuffle service with a yarn mini-cluster
+ */
+@ExtendedYarnTest
+class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite {
+
+  override def newYarnConfig(): YarnConfiguration = {
+    val yarnConfig = new YarnConfiguration()
+    yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle")
+    yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"),
+      classOf[YarnShuffleService].getCanonicalName)
+    yarnConfig.set("spark.shuffle.service.port", "0")
+    yarnConfig
+  }
+
+  test("external shuffle service") {
+    val shuffleServicePort = YarnTestAccessor.getShuffleServicePort
+    val shuffleService = YarnTestAccessor.getShuffleServiceInstance
+
+    val registeredExecFile = YarnTestAccessor.getRegisteredExecutorFile(shuffleService)
+
+    logInfo("Shuffle service port = " + shuffleServicePort)
+    val result = File.createTempFile("result", null, tempDir)
+    val finalState = runSpark(
+      false,
+      mainClassName(YarnExternalShuffleDriver.getClass),
+      appArgs = Seq(result.getAbsolutePath(), registeredExecFile.getAbsolutePath),
+      extraConf = Map(
+        "spark.shuffle.service.enabled" -> "true",
+        "spark.shuffle.service.port" -> shuffleServicePort.toString
+      )
+    )
+    checkResult(finalState, result)
+    assert(YarnTestAccessor.getRegisteredExecutorFile(shuffleService).exists())
+  }
+}
+
+private object YarnExternalShuffleDriver extends Logging with Matchers {
+
+  val WAIT_TIMEOUT_MILLIS = 10000
+
+  def main(args: Array[String]): Unit = {
+    if (args.length != 2) {
+      // scalastyle:off println
+      System.err.println(
+        s"""
+        |Invalid command line: ${args.mkString(" ")}
+        |
+        |Usage: ExternalShuffleDriver [result file] [registered exec file]
+        """.stripMargin)
+      // scalastyle:on println
+      System.exit(1)
+    }
+
+    val sc = new SparkContext(new SparkConf()
+      .setAppName("External Shuffle Test"))
+    val conf = sc.getConf
+    val status = new File(args(0))
+    val registeredExecFile = new File(args(1))
+    logInfo("shuffle service executor file = " + registeredExecFile)
+    var result = "failure"
+    val execStateCopy = new File(registeredExecFile.getAbsolutePath + "_dup")
+    try {
+      val data = sc.parallelize(0 until 100, 10).map { x => (x % 10) -> x }.reduceByKey{ _ + _ }.
+        collect().toSet
+      sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
+      data should be ((0 until 10).map{x => x -> (x * 10 + 450)}.toSet)
+      result = "success"
+      // only one process can open a leveldb file at a time, so we copy the files
+      FileUtils.copyDirectory(registeredExecFile, execStateCopy)
+      assert(!ShuffleTestAccessor.reloadRegisteredExecutors(execStateCopy).isEmpty)
+    } finally {
+      sc.stop()
+      FileUtils.deleteDirectory(execStateCopy)
+      Files.write(result, status, StandardCharsets.UTF_8)
+    }
+  }
+
+}


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


[17/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala
deleted file mode 100644
index f384290..0000000
--- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala
+++ /dev/null
@@ -1,740 +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.mesos
-
-import java.io.File
-import java.util.{Collections, Date, List => JList}
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.mesos.{Scheduler, SchedulerDriver}
-import org.apache.mesos.Protos.{TaskState => MesosTaskState, _}
-import org.apache.mesos.Protos.Environment.Variable
-import org.apache.mesos.Protos.TaskStatus.Reason
-
-import org.apache.spark.{SecurityManager, SparkConf, SparkException, TaskState}
-import org.apache.spark.deploy.mesos.MesosDriverDescription
-import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse}
-import org.apache.spark.metrics.MetricsSystem
-import org.apache.spark.util.Utils
-
-/**
- * Tracks the current state of a Mesos Task that runs a Spark driver.
- * @param driverDescription Submitted driver description from
- * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]]
- * @param taskId Mesos TaskID generated for the task
- * @param slaveId Slave ID that the task is assigned to
- * @param mesosTaskStatus The last known task status update.
- * @param startDate The date the task was launched
- * @param finishDate The date the task finished
- * @param frameworkId Mesos framework ID the task registers with
- */
-private[spark] class MesosClusterSubmissionState(
-    val driverDescription: MesosDriverDescription,
-    val taskId: TaskID,
-    val slaveId: SlaveID,
-    var mesosTaskStatus: Option[TaskStatus],
-    var startDate: Date,
-    var finishDate: Option[Date],
-    val frameworkId: String)
-  extends Serializable {
-
-  def copy(): MesosClusterSubmissionState = {
-    new MesosClusterSubmissionState(
-      driverDescription, taskId, slaveId, mesosTaskStatus, startDate, finishDate, frameworkId)
-  }
-}
-
-/**
- * Tracks the retry state of a driver, which includes the next time it should be scheduled
- * and necessary information to do exponential backoff.
- * This class is not thread-safe, and we expect the caller to handle synchronizing state.
- *
- * @param lastFailureStatus Last Task status when it failed.
- * @param retries Number of times it has been retried.
- * @param nextRetry Time at which it should be retried next
- * @param waitTime The amount of time driver is scheduled to wait until next retry.
- */
-private[spark] class MesosClusterRetryState(
-    val lastFailureStatus: TaskStatus,
-    val retries: Int,
-    val nextRetry: Date,
-    val waitTime: Int) extends Serializable {
-  def copy(): MesosClusterRetryState =
-    new MesosClusterRetryState(lastFailureStatus, retries, nextRetry, waitTime)
-}
-
-/**
- * The full state of the cluster scheduler, currently being used for displaying
- * information on the UI.
- *
- * @param frameworkId Mesos Framework id for the cluster scheduler.
- * @param masterUrl The Mesos master url
- * @param queuedDrivers All drivers queued to be launched
- * @param launchedDrivers All launched or running drivers
- * @param finishedDrivers All terminated drivers
- * @param pendingRetryDrivers All drivers pending to be retried
- */
-private[spark] class MesosClusterSchedulerState(
-    val frameworkId: String,
-    val masterUrl: Option[String],
-    val queuedDrivers: Iterable[MesosDriverDescription],
-    val launchedDrivers: Iterable[MesosClusterSubmissionState],
-    val finishedDrivers: Iterable[MesosClusterSubmissionState],
-    val pendingRetryDrivers: Iterable[MesosDriverDescription])
-
-/**
- * The full state of a Mesos driver, that is being used to display driver information on the UI.
- */
-private[spark] class MesosDriverState(
-    val state: String,
-    val description: MesosDriverDescription,
-    val submissionState: Option[MesosClusterSubmissionState] = None)
-
-/**
- * A Mesos scheduler that is responsible for launching submitted Spark drivers in cluster mode
- * as Mesos tasks in a Mesos cluster.
- * All drivers are launched asynchronously by the framework, which will eventually be launched
- * by one of the slaves in the cluster. The results of the driver will be stored in slave's task
- * sandbox which is accessible by visiting the Mesos UI.
- * This scheduler supports recovery by persisting all its state and performs task reconciliation
- * on recover, which gets all the latest state for all the drivers from Mesos master.
- */
-private[spark] class MesosClusterScheduler(
-    engineFactory: MesosClusterPersistenceEngineFactory,
-    conf: SparkConf)
-  extends Scheduler with MesosSchedulerUtils {
-  var frameworkUrl: String = _
-  private val metricsSystem =
-    MetricsSystem.createMetricsSystem("mesos_cluster", conf, new SecurityManager(conf))
-  private val master = conf.get("spark.master")
-  private val appName = conf.get("spark.app.name")
-  private val queuedCapacity = conf.getInt("spark.mesos.maxDrivers", 200)
-  private val retainedDrivers = conf.getInt("spark.mesos.retainedDrivers", 200)
-  private val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute
-  private val useFetchCache = conf.getBoolean("spark.mesos.fetchCache.enable", false)
-  private val schedulerState = engineFactory.createEngine("scheduler")
-  private val stateLock = new Object()
-  private val finishedDrivers =
-    new mutable.ArrayBuffer[MesosClusterSubmissionState](retainedDrivers)
-  private var frameworkId: String = null
-  // Holds all the launched drivers and current launch state, keyed by driver id.
-  private val launchedDrivers = new mutable.HashMap[String, MesosClusterSubmissionState]()
-  // Holds a map of driver id to expected slave id that is passed to Mesos for reconciliation.
-  // All drivers that are loaded after failover are added here, as we need get the latest
-  // state of the tasks from Mesos.
-  private val pendingRecover = new mutable.HashMap[String, SlaveID]()
-  // Stores all the submitted drivers that hasn't been launched.
-  private val queuedDrivers = new ArrayBuffer[MesosDriverDescription]()
-  // All supervised drivers that are waiting to retry after termination.
-  private val pendingRetryDrivers = new ArrayBuffer[MesosDriverDescription]()
-  private val queuedDriversState = engineFactory.createEngine("driverQueue")
-  private val launchedDriversState = engineFactory.createEngine("launchedDrivers")
-  private val pendingRetryDriversState = engineFactory.createEngine("retryList")
-  // Flag to mark if the scheduler is ready to be called, which is until the scheduler
-  // is registered with Mesos master.
-  @volatile protected var ready = false
-  private var masterInfo: Option[MasterInfo] = None
-
-  def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse = {
-    val c = new CreateSubmissionResponse
-    if (!ready) {
-      c.success = false
-      c.message = "Scheduler is not ready to take requests"
-      return c
-    }
-
-    stateLock.synchronized {
-      if (isQueueFull()) {
-        c.success = false
-        c.message = "Already reached maximum submission size"
-        return c
-      }
-      c.submissionId = desc.submissionId
-      queuedDriversState.persist(desc.submissionId, desc)
-      queuedDrivers += desc
-      c.success = true
-    }
-    c
-  }
-
-  def killDriver(submissionId: String): KillSubmissionResponse = {
-    val k = new KillSubmissionResponse
-    if (!ready) {
-      k.success = false
-      k.message = "Scheduler is not ready to take requests"
-      return k
-    }
-    k.submissionId = submissionId
-    stateLock.synchronized {
-      // We look for the requested driver in the following places:
-      // 1. Check if submission is running or launched.
-      // 2. Check if it's still queued.
-      // 3. Check if it's in the retry list.
-      // 4. Check if it has already completed.
-      if (launchedDrivers.contains(submissionId)) {
-        val task = launchedDrivers(submissionId)
-        mesosDriver.killTask(task.taskId)
-        k.success = true
-        k.message = "Killing running driver"
-      } else if (removeFromQueuedDrivers(submissionId)) {
-        k.success = true
-        k.message = "Removed driver while it's still pending"
-      } else if (removeFromPendingRetryDrivers(submissionId)) {
-        k.success = true
-        k.message = "Removed driver while it's being retried"
-      } else if (finishedDrivers.exists(_.driverDescription.submissionId.equals(submissionId))) {
-        k.success = false
-        k.message = "Driver already terminated"
-      } else {
-        k.success = false
-        k.message = "Cannot find driver"
-      }
-    }
-    k
-  }
-
-  def getDriverStatus(submissionId: String): SubmissionStatusResponse = {
-    val s = new SubmissionStatusResponse
-    if (!ready) {
-      s.success = false
-      s.message = "Scheduler is not ready to take requests"
-      return s
-    }
-    s.submissionId = submissionId
-    stateLock.synchronized {
-      if (queuedDrivers.exists(_.submissionId.equals(submissionId))) {
-        s.success = true
-        s.driverState = "QUEUED"
-      } else if (launchedDrivers.contains(submissionId)) {
-        s.success = true
-        s.driverState = "RUNNING"
-        launchedDrivers(submissionId).mesosTaskStatus.foreach(state => s.message = state.toString)
-      } else if (finishedDrivers.exists(_.driverDescription.submissionId.equals(submissionId))) {
-        s.success = true
-        s.driverState = "FINISHED"
-        finishedDrivers
-          .find(d => d.driverDescription.submissionId.equals(submissionId)).get.mesosTaskStatus
-          .foreach(state => s.message = state.toString)
-      } else if (pendingRetryDrivers.exists(_.submissionId.equals(submissionId))) {
-        val status = pendingRetryDrivers.find(_.submissionId.equals(submissionId))
-          .get.retryState.get.lastFailureStatus
-        s.success = true
-        s.driverState = "RETRYING"
-        s.message = status.toString
-      } else {
-        s.success = false
-        s.driverState = "NOT_FOUND"
-      }
-    }
-    s
-  }
-
-  /**
-   * Gets the driver state to be displayed on the Web UI.
-   */
-  def getDriverState(submissionId: String): Option[MesosDriverState] = {
-    stateLock.synchronized {
-      queuedDrivers.find(_.submissionId.equals(submissionId))
-        .map(d => new MesosDriverState("QUEUED", d))
-        .orElse(launchedDrivers.get(submissionId)
-          .map(d => new MesosDriverState("RUNNING", d.driverDescription, Some(d))))
-        .orElse(finishedDrivers.find(_.driverDescription.submissionId.equals(submissionId))
-          .map(d => new MesosDriverState("FINISHED", d.driverDescription, Some(d))))
-        .orElse(pendingRetryDrivers.find(_.submissionId.equals(submissionId))
-          .map(d => new MesosDriverState("RETRYING", d)))
-    }
-  }
-
-  private def isQueueFull(): Boolean = launchedDrivers.size >= queuedCapacity
-
-  /**
-   * Recover scheduler state that is persisted.
-   * We still need to do task reconciliation to be up to date of the latest task states
-   * as it might have changed while the scheduler is failing over.
-   */
-  private def recoverState(): Unit = {
-    stateLock.synchronized {
-      launchedDriversState.fetchAll[MesosClusterSubmissionState]().foreach { state =>
-        launchedDrivers(state.taskId.getValue) = state
-        pendingRecover(state.taskId.getValue) = state.slaveId
-      }
-      queuedDriversState.fetchAll[MesosDriverDescription]().foreach(d => queuedDrivers += d)
-      // There is potential timing issue where a queued driver might have been launched
-      // but the scheduler shuts down before the queued driver was able to be removed
-      // from the queue. We try to mitigate this issue by walking through all queued drivers
-      // and remove if they're already launched.
-      queuedDrivers
-        .filter(d => launchedDrivers.contains(d.submissionId))
-        .foreach(d => removeFromQueuedDrivers(d.submissionId))
-      pendingRetryDriversState.fetchAll[MesosDriverDescription]()
-        .foreach(s => pendingRetryDrivers += s)
-      // TODO: Consider storing finished drivers so we can show them on the UI after
-      // failover. For now we clear the history on each recovery.
-      finishedDrivers.clear()
-    }
-  }
-
-  /**
-   * Starts the cluster scheduler and wait until the scheduler is registered.
-   * This also marks the scheduler to be ready for requests.
-   */
-  def start(): Unit = {
-    // TODO: Implement leader election to make sure only one framework running in the cluster.
-    val fwId = schedulerState.fetch[String]("frameworkId")
-    fwId.foreach { id =>
-      frameworkId = id
-    }
-    recoverState()
-    metricsSystem.registerSource(new MesosClusterSchedulerSource(this))
-    metricsSystem.start()
-    val driver = createSchedulerDriver(
-      master,
-      MesosClusterScheduler.this,
-      Utils.getCurrentUserName(),
-      appName,
-      conf,
-      Some(frameworkUrl),
-      Some(true),
-      Some(Integer.MAX_VALUE),
-      fwId)
-
-    startScheduler(driver)
-    ready = true
-  }
-
-  def stop(): Unit = {
-    ready = false
-    metricsSystem.report()
-    metricsSystem.stop()
-    mesosDriver.stop(true)
-  }
-
-  override def registered(
-      driver: SchedulerDriver,
-      newFrameworkId: FrameworkID,
-      masterInfo: MasterInfo): Unit = {
-    logInfo("Registered as framework ID " + newFrameworkId.getValue)
-    if (newFrameworkId.getValue != frameworkId) {
-      frameworkId = newFrameworkId.getValue
-      schedulerState.persist("frameworkId", frameworkId)
-    }
-    markRegistered()
-
-    stateLock.synchronized {
-      this.masterInfo = Some(masterInfo)
-      if (!pendingRecover.isEmpty) {
-        // Start task reconciliation if we need to recover.
-        val statuses = pendingRecover.collect {
-          case (taskId, slaveId) =>
-            val newStatus = TaskStatus.newBuilder()
-              .setTaskId(TaskID.newBuilder().setValue(taskId).build())
-              .setSlaveId(slaveId)
-              .setState(MesosTaskState.TASK_STAGING)
-              .build()
-            launchedDrivers.get(taskId).map(_.mesosTaskStatus.getOrElse(newStatus))
-              .getOrElse(newStatus)
-        }
-        // TODO: Page the status updates to avoid trying to reconcile
-        // a large amount of tasks at once.
-        driver.reconcileTasks(statuses.toSeq.asJava)
-      }
-    }
-  }
-
-  private def getDriverExecutorURI(desc: MesosDriverDescription): Option[String] = {
-    desc.conf.getOption("spark.executor.uri")
-      .orElse(desc.command.environment.get("SPARK_EXECUTOR_URI"))
-  }
-
-  private def getDriverFrameworkID(desc: MesosDriverDescription): String = {
-    s"${frameworkId}-${desc.submissionId}"
-  }
-
-  private def adjust[A, B](m: collection.Map[A, B], k: A, default: B)(f: B => B) = {
-    m.updated(k, f(m.getOrElse(k, default)))
-  }
-
-  private def getDriverEnvironment(desc: MesosDriverDescription): Environment = {
-    // TODO(mgummelt): Don't do this here.  This should be passed as a --conf
-    val commandEnv = adjust(desc.command.environment, "SPARK_SUBMIT_OPTS", "")(
-      v => s"$v -Dspark.mesos.driver.frameworkId=${getDriverFrameworkID(desc)}"
-    )
-
-    val env = desc.conf.getAllWithPrefix("spark.mesos.driverEnv.") ++ commandEnv
-
-    val envBuilder = Environment.newBuilder()
-    env.foreach { case (k, v) =>
-      envBuilder.addVariables(Variable.newBuilder().setName(k).setValue(v))
-    }
-    envBuilder.build()
-  }
-
-  private def getDriverUris(desc: MesosDriverDescription): List[CommandInfo.URI] = {
-    val confUris = List(conf.getOption("spark.mesos.uris"),
-      desc.conf.getOption("spark.mesos.uris"),
-      desc.conf.getOption("spark.submit.pyFiles")).flatMap(
-      _.map(_.split(",").map(_.trim))
-    ).flatten
-
-    val jarUrl = desc.jarUrl.stripPrefix("file:").stripPrefix("local:")
-
-    ((jarUrl :: confUris) ++ getDriverExecutorURI(desc).toList).map(uri =>
-      CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetchCache).build())
-  }
-
-  private def getDriverCommandValue(desc: MesosDriverDescription): String = {
-    val dockerDefined = desc.conf.contains("spark.mesos.executor.docker.image")
-    val executorUri = getDriverExecutorURI(desc)
-    // Gets the path to run spark-submit, and the path to the Mesos sandbox.
-    val (executable, sandboxPath) = if (dockerDefined) {
-      // Application jar is automatically downloaded in the mounted sandbox by Mesos,
-      // and the path to the mounted volume is stored in $MESOS_SANDBOX env variable.
-      ("./bin/spark-submit", "$MESOS_SANDBOX")
-    } else if (executorUri.isDefined) {
-      val folderBasename = executorUri.get.split('/').last.split('.').head
-
-      val entries = conf.getOption("spark.executor.extraLibraryPath")
-        .map(path => Seq(path) ++ desc.command.libraryPathEntries)
-        .getOrElse(desc.command.libraryPathEntries)
-
-      val prefixEnv = if (!entries.isEmpty) Utils.libraryPathEnvPrefix(entries) else ""
-
-      val cmdExecutable = s"cd $folderBasename*; $prefixEnv bin/spark-submit"
-      // Sandbox path points to the parent folder as we chdir into the folderBasename.
-      (cmdExecutable, "..")
-    } else {
-      val executorSparkHome = desc.conf.getOption("spark.mesos.executor.home")
-        .orElse(conf.getOption("spark.home"))
-        .orElse(Option(System.getenv("SPARK_HOME")))
-        .getOrElse {
-          throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!")
-        }
-      val cmdExecutable = new File(executorSparkHome, "./bin/spark-submit").getPath
-      // Sandbox points to the current directory by default with Mesos.
-      (cmdExecutable, ".")
-    }
-    val cmdOptions = generateCmdOption(desc, sandboxPath).mkString(" ")
-    val primaryResource = new File(sandboxPath, desc.jarUrl.split("/").last).toString()
-    val appArguments = desc.command.arguments.mkString(" ")
-
-    s"$executable $cmdOptions $primaryResource $appArguments"
-  }
-
-  private def buildDriverCommand(desc: MesosDriverDescription): CommandInfo = {
-    val builder = CommandInfo.newBuilder()
-    builder.setValue(getDriverCommandValue(desc))
-    builder.setEnvironment(getDriverEnvironment(desc))
-    builder.addAllUris(getDriverUris(desc).asJava)
-    builder.build()
-  }
-
-  private def generateCmdOption(desc: MesosDriverDescription, sandboxPath: String): Seq[String] = {
-    var options = Seq(
-      "--name", desc.conf.get("spark.app.name"),
-      "--master", s"mesos://${conf.get("spark.master")}",
-      "--driver-cores", desc.cores.toString,
-      "--driver-memory", s"${desc.mem}M")
-
-    // Assume empty main class means we're running python
-    if (!desc.command.mainClass.equals("")) {
-      options ++= Seq("--class", desc.command.mainClass)
-    }
-
-    desc.conf.getOption("spark.executor.memory").foreach { v =>
-      options ++= Seq("--executor-memory", v)
-    }
-    desc.conf.getOption("spark.cores.max").foreach { v =>
-      options ++= Seq("--total-executor-cores", v)
-    }
-    desc.conf.getOption("spark.submit.pyFiles").foreach { pyFiles =>
-      val formattedFiles = pyFiles.split(",")
-        .map { path => new File(sandboxPath, path.split("/").last).toString() }
-        .mkString(",")
-      options ++= Seq("--py-files", formattedFiles)
-    }
-
-    // --conf
-    val replicatedOptionsBlacklist = Set(
-      "spark.jars", // Avoids duplicate classes in classpath
-      "spark.submit.deployMode", // this would be set to `cluster`, but we need client
-      "spark.master" // this contains the address of the dispatcher, not master
-    )
-    val defaultConf = conf.getAllWithPrefix("spark.mesos.dispatcher.driverDefault.").toMap
-    val driverConf = desc.conf.getAll
-      .filter { case (key, _) => !replicatedOptionsBlacklist.contains(key) }
-      .toMap
-    (defaultConf ++ driverConf).foreach { case (key, value) =>
-      options ++= Seq("--conf", s""""$key=${shellEscape(value)}"""".stripMargin) }
-
-    options
-  }
-
-  /**
-   * Escape args for Unix-like shells, unless already quoted by the user.
-   * Based on: http://www.gnu.org/software/bash/manual/html_node/Double-Quotes.html
-   * and http://www.grymoire.com/Unix/Quote.html
- *
-   * @param value argument
-   * @return escaped argument
-   */
-  private[scheduler] def shellEscape(value: String): String = {
-    val WrappedInQuotes = """^(".+"|'.+')$""".r
-    val ShellSpecialChars = (""".*([ '<>&|\?\*;!#\\(\)"$`]).*""").r
-    value match {
-      case WrappedInQuotes(c) => value // The user quoted his args, don't touch it!
-      case ShellSpecialChars(c) => "\"" + value.replaceAll("""(["`\$\\])""", """\\$1""") + "\""
-      case _: String => value // Don't touch harmless strings
-    }
-  }
-
-  private class ResourceOffer(
-      val offerId: OfferID,
-      val slaveId: SlaveID,
-      var resources: JList[Resource]) {
-    override def toString(): String = {
-      s"Offer id: ${offerId}, resources: ${resources}"
-    }
-  }
-
-  private def createTaskInfo(desc: MesosDriverDescription, offer: ResourceOffer): TaskInfo = {
-    val taskId = TaskID.newBuilder().setValue(desc.submissionId).build()
-
-    val (remainingResources, cpuResourcesToUse) =
-      partitionResources(offer.resources, "cpus", desc.cores)
-    val (finalResources, memResourcesToUse) =
-      partitionResources(remainingResources.asJava, "mem", desc.mem)
-    offer.resources = finalResources.asJava
-
-    val appName = desc.conf.get("spark.app.name")
-    val taskInfo = TaskInfo.newBuilder()
-      .setTaskId(taskId)
-      .setName(s"Driver for ${appName}")
-      .setSlaveId(offer.slaveId)
-      .setCommand(buildDriverCommand(desc))
-      .addAllResources(cpuResourcesToUse.asJava)
-      .addAllResources(memResourcesToUse.asJava)
-    taskInfo.setContainer(MesosSchedulerBackendUtil.containerInfo(desc.conf))
-    taskInfo.build
-  }
-
-  /**
-   * This method takes all the possible candidates and attempt to schedule them with Mesos offers.
-   * Every time a new task is scheduled, the afterLaunchCallback is called to perform post scheduled
-   * logic on each task.
-   */
-  private def scheduleTasks(
-      candidates: Seq[MesosDriverDescription],
-      afterLaunchCallback: (String) => Boolean,
-      currentOffers: List[ResourceOffer],
-      tasks: mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]): Unit = {
-    for (submission <- candidates) {
-      val driverCpu = submission.cores
-      val driverMem = submission.mem
-      logTrace(s"Finding offer to launch driver with cpu: $driverCpu, mem: $driverMem")
-      val offerOption = currentOffers.find { o =>
-        getResource(o.resources, "cpus") >= driverCpu &&
-        getResource(o.resources, "mem") >= driverMem
-      }
-      if (offerOption.isEmpty) {
-        logDebug(s"Unable to find offer to launch driver id: ${submission.submissionId}, " +
-          s"cpu: $driverCpu, mem: $driverMem")
-      } else {
-        val offer = offerOption.get
-        val queuedTasks = tasks.getOrElseUpdate(offer.offerId, new ArrayBuffer[TaskInfo])
-        val task = createTaskInfo(submission, offer)
-        queuedTasks += task
-        logTrace(s"Using offer ${offer.offerId.getValue} to launch driver " +
-          submission.submissionId)
-        val newState = new MesosClusterSubmissionState(submission, task.getTaskId, offer.slaveId,
-          None, new Date(), None, getDriverFrameworkID(submission))
-        launchedDrivers(submission.submissionId) = newState
-        launchedDriversState.persist(submission.submissionId, newState)
-        afterLaunchCallback(submission.submissionId)
-      }
-    }
-  }
-
-  override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = {
-    logTrace(s"Received offers from Mesos: \n${offers.asScala.mkString("\n")}")
-    val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]()
-    val currentTime = new Date()
-
-    val currentOffers = offers.asScala.map {
-      o => new ResourceOffer(o.getId, o.getSlaveId, o.getResourcesList)
-    }.toList
-
-    stateLock.synchronized {
-      // We first schedule all the supervised drivers that are ready to retry.
-      // This list will be empty if none of the drivers are marked as supervise.
-      val driversToRetry = pendingRetryDrivers.filter { d =>
-        d.retryState.get.nextRetry.before(currentTime)
-      }
-
-      scheduleTasks(
-        copyBuffer(driversToRetry),
-        removeFromPendingRetryDrivers,
-        currentOffers,
-        tasks)
-
-      // Then we walk through the queued drivers and try to schedule them.
-      scheduleTasks(
-        copyBuffer(queuedDrivers),
-        removeFromQueuedDrivers,
-        currentOffers,
-        tasks)
-    }
-    tasks.foreach { case (offerId, taskInfos) =>
-      driver.launchTasks(Collections.singleton(offerId), taskInfos.asJava)
-    }
-
-    for (o <- currentOffers if !tasks.contains(o.offerId)) {
-      driver.declineOffer(o.offerId)
-    }
-  }
-
-  private def copyBuffer(
-      buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = {
-    val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size)
-    buffer.copyToBuffer(newBuffer)
-    newBuffer
-  }
-
-  def getSchedulerState(): MesosClusterSchedulerState = {
-    stateLock.synchronized {
-      new MesosClusterSchedulerState(
-        frameworkId,
-        masterInfo.map(m => s"http://${m.getIp}:${m.getPort}"),
-        copyBuffer(queuedDrivers),
-        launchedDrivers.values.map(_.copy()).toList,
-        finishedDrivers.map(_.copy()).toList,
-        copyBuffer(pendingRetryDrivers))
-    }
-  }
-
-  override def offerRescinded(driver: SchedulerDriver, offerId: OfferID): Unit = {}
-  override def disconnected(driver: SchedulerDriver): Unit = {}
-  override def reregistered(driver: SchedulerDriver, masterInfo: MasterInfo): Unit = {
-    logInfo(s"Framework re-registered with master ${masterInfo.getId}")
-  }
-  override def slaveLost(driver: SchedulerDriver, slaveId: SlaveID): Unit = {}
-  override def error(driver: SchedulerDriver, error: String): Unit = {
-    logError("Error received: " + error)
-    markErr()
-  }
-
-  /**
-   * Check if the task state is a recoverable state that we can relaunch the task.
-   * Task state like TASK_ERROR are not relaunchable state since it wasn't able
-   * to be validated by Mesos.
-   */
-  private def shouldRelaunch(state: MesosTaskState): Boolean = {
-    state == MesosTaskState.TASK_FAILED ||
-      state == MesosTaskState.TASK_KILLED ||
-      state == MesosTaskState.TASK_LOST
-  }
-
-  override def statusUpdate(driver: SchedulerDriver, status: TaskStatus): Unit = {
-    val taskId = status.getTaskId.getValue
-    stateLock.synchronized {
-      if (launchedDrivers.contains(taskId)) {
-        if (status.getReason == Reason.REASON_RECONCILIATION &&
-          !pendingRecover.contains(taskId)) {
-          // Task has already received update and no longer requires reconciliation.
-          return
-        }
-        val state = launchedDrivers(taskId)
-        // Check if the driver is supervise enabled and can be relaunched.
-        if (state.driverDescription.supervise && shouldRelaunch(status.getState)) {
-          removeFromLaunchedDrivers(taskId)
-          state.finishDate = Some(new Date())
-          val retryState: Option[MesosClusterRetryState] = state.driverDescription.retryState
-          val (retries, waitTimeSec) = retryState
-            .map { rs => (rs.retries + 1, Math.min(maxRetryWaitTime, rs.waitTime * 2)) }
-            .getOrElse{ (1, 1) }
-          val nextRetry = new Date(new Date().getTime + waitTimeSec * 1000L)
-
-          val newDriverDescription = state.driverDescription.copy(
-            retryState = Some(new MesosClusterRetryState(status, retries, nextRetry, waitTimeSec)))
-          pendingRetryDrivers += newDriverDescription
-          pendingRetryDriversState.persist(taskId, newDriverDescription)
-        } else if (TaskState.isFinished(mesosToTaskState(status.getState))) {
-          removeFromLaunchedDrivers(taskId)
-          state.finishDate = Some(new Date())
-          if (finishedDrivers.size >= retainedDrivers) {
-            val toRemove = math.max(retainedDrivers / 10, 1)
-            finishedDrivers.trimStart(toRemove)
-          }
-          finishedDrivers += state
-        }
-        state.mesosTaskStatus = Option(status)
-      } else {
-        logError(s"Unable to find driver $taskId in status update")
-      }
-    }
-  }
-
-  override def frameworkMessage(
-      driver: SchedulerDriver,
-      executorId: ExecutorID,
-      slaveId: SlaveID,
-      message: Array[Byte]): Unit = {}
-
-  override def executorLost(
-      driver: SchedulerDriver,
-      executorId: ExecutorID,
-      slaveId: SlaveID,
-      status: Int): Unit = {}
-
-  private def removeFromQueuedDrivers(id: String): Boolean = {
-    val index = queuedDrivers.indexWhere(_.submissionId.equals(id))
-    if (index != -1) {
-      queuedDrivers.remove(index)
-      queuedDriversState.expunge(id)
-      true
-    } else {
-      false
-    }
-  }
-
-  private def removeFromLaunchedDrivers(id: String): Boolean = {
-    if (launchedDrivers.remove(id).isDefined) {
-      launchedDriversState.expunge(id)
-      true
-    } else {
-      false
-    }
-  }
-
-  private def removeFromPendingRetryDrivers(id: String): Boolean = {
-    val index = pendingRetryDrivers.indexWhere(_.submissionId.equals(id))
-    if (index != -1) {
-      pendingRetryDrivers.remove(index)
-      pendingRetryDriversState.expunge(id)
-      true
-    } else {
-      false
-    }
-  }
-
-  def getQueuedDriversSize: Int = queuedDrivers.size
-  def getLaunchedDriversSize: Int = launchedDrivers.size
-  def getPendingRetryDriversSize: Int = pendingRetryDrivers.size
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala
deleted file mode 100644
index 1fe9497..0000000
--- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala
+++ /dev/null
@@ -1,40 +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.mesos
-
-import com.codahale.metrics.{Gauge, MetricRegistry}
-
-import org.apache.spark.metrics.source.Source
-
-private[mesos] class MesosClusterSchedulerSource(scheduler: MesosClusterScheduler)
-  extends Source {
-  override def sourceName: String = "mesos_cluster"
-  override def metricRegistry: MetricRegistry = new MetricRegistry()
-
-  metricRegistry.register(MetricRegistry.name("waitingDrivers"), new Gauge[Int] {
-    override def getValue: Int = scheduler.getQueuedDriversSize
-  })
-
-  metricRegistry.register(MetricRegistry.name("launchedDrivers"), new Gauge[Int] {
-    override def getValue: Int = scheduler.getLaunchedDriversSize
-  })
-
-  metricRegistry.register(MetricRegistry.name("retryDrivers"), new Gauge[Int] {
-    override def getValue: Int = scheduler.getPendingRetryDriversSize
-  })
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
deleted file mode 100644
index 3258b09..0000000
--- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
+++ /dev/null
@@ -1,668 +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.mesos
-
-import java.io.File
-import java.util.{Collections, List => JList}
-import java.util.concurrent.locks.ReentrantLock
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-import scala.concurrent.Future
-
-import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _}
-
-import org.apache.spark.{SecurityManager, SparkContext, SparkException, TaskState}
-import org.apache.spark.network.netty.SparkTransportConf
-import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient
-import org.apache.spark.rpc.RpcEndpointAddress
-import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl}
-import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
-import org.apache.spark.util.Utils
-
-/**
- * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds
- * onto each Mesos node for the duration of the Spark job instead of relinquishing cores whenever
- * a task is done. It launches Spark tasks within the coarse-grained Mesos tasks using the
- * CoarseGrainedSchedulerBackend mechanism. This class is useful for lower and more predictable
- * latency.
- *
- * Unfortunately this has a bit of duplication from [[MesosFineGrainedSchedulerBackend]],
- * but it seems hard to remove this.
- */
-private[spark] class MesosCoarseGrainedSchedulerBackend(
-    scheduler: TaskSchedulerImpl,
-    sc: SparkContext,
-    master: String,
-    securityManager: SecurityManager)
-  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv)
-  with org.apache.mesos.Scheduler
-  with MesosSchedulerUtils {
-
-  val MAX_SLAVE_FAILURES = 2     // Blacklist a slave after this many failures
-
-  // Maximum number of cores to acquire (TODO: we'll need more flexible controls here)
-  val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt
-
-  val useFetcherCache = conf.getBoolean("spark.mesos.fetcherCache.enable", false)
-
-  val maxGpus = conf.getInt("spark.mesos.gpus.max", 0)
-
-  private[this] val shutdownTimeoutMS =
-    conf.getTimeAsMs("spark.mesos.coarse.shutdownTimeout", "10s")
-      .ensuring(_ >= 0, "spark.mesos.coarse.shutdownTimeout must be >= 0")
-
-  // Synchronization protected by stateLock
-  private[this] var stopCalled: Boolean = false
-
-  // If shuffle service is enabled, the Spark driver will register with the shuffle service.
-  // This is for cleaning up shuffle files reliably.
-  private val shuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false)
-
-  // Cores we have acquired with each Mesos task ID
-  val coresByTaskId = new mutable.HashMap[String, Int]
-  val gpusByTaskId = new mutable.HashMap[String, Int]
-  var totalCoresAcquired = 0
-  var totalGpusAcquired = 0
-
-  // SlaveID -> Slave
-  // This map accumulates entries for the duration of the job.  Slaves are never deleted, because
-  // we need to maintain e.g. failure state and connection state.
-  private val slaves = new mutable.HashMap[String, Slave]
-
-  /**
-   * The total number of executors we aim to have. Undefined when not using dynamic allocation.
-   * Initially set to 0 when using dynamic allocation, the executor allocation manager will send
-   * the real initial limit later.
-   */
-  private var executorLimitOption: Option[Int] = {
-    if (Utils.isDynamicAllocationEnabled(conf)) {
-      Some(0)
-    } else {
-      None
-    }
-  }
-
-  /**
-   *  Return the current executor limit, which may be [[Int.MaxValue]]
-   *  before properly initialized.
-   */
-  private[mesos] def executorLimit: Int = executorLimitOption.getOrElse(Int.MaxValue)
-
-  // private lock object protecting mutable state above. Using the intrinsic lock
-  // may lead to deadlocks since the superclass might also try to lock
-  private val stateLock = new ReentrantLock
-
-  val extraCoresPerExecutor = conf.getInt("spark.mesos.extra.cores", 0)
-
-  // Offer constraints
-  private val slaveOfferConstraints =
-    parseConstraintString(sc.conf.get("spark.mesos.constraints", ""))
-
-  // Reject offers with mismatched constraints in seconds
-  private val rejectOfferDurationForUnmetConstraints =
-    getRejectOfferDurationForUnmetConstraints(sc)
-
-  // Reject offers when we reached the maximum number of cores for this framework
-  private val rejectOfferDurationForReachedMaxCores =
-    getRejectOfferDurationForReachedMaxCores(sc)
-
-  // A client for talking to the external shuffle service
-  private val mesosExternalShuffleClient: Option[MesosExternalShuffleClient] = {
-    if (shuffleServiceEnabled) {
-      Some(getShuffleClient())
-    } else {
-      None
-    }
-  }
-
-  // This method is factored out for testability
-  protected def getShuffleClient(): MesosExternalShuffleClient = {
-    new MesosExternalShuffleClient(
-      SparkTransportConf.fromSparkConf(conf, "shuffle"),
-      securityManager,
-      securityManager.isAuthenticationEnabled(),
-      securityManager.isSaslEncryptionEnabled())
-  }
-
-  var nextMesosTaskId = 0
-
-  @volatile var appId: String = _
-
-  def newMesosTaskId(): String = {
-    val id = nextMesosTaskId
-    nextMesosTaskId += 1
-    id.toString
-  }
-
-  override def start() {
-    super.start()
-    val driver = createSchedulerDriver(
-      master,
-      MesosCoarseGrainedSchedulerBackend.this,
-      sc.sparkUser,
-      sc.appName,
-      sc.conf,
-      sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.webUrl)),
-      None,
-      None,
-      sc.conf.getOption("spark.mesos.driver.frameworkId")
-    )
-
-    unsetFrameworkID(sc)
-    startScheduler(driver)
-  }
-
-  def createCommand(offer: Offer, numCores: Int, taskId: String): CommandInfo = {
-    val environment = Environment.newBuilder()
-    val extraClassPath = conf.getOption("spark.executor.extraClassPath")
-    extraClassPath.foreach { cp =>
-      environment.addVariables(
-        Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build())
-    }
-    val extraJavaOpts = conf.get("spark.executor.extraJavaOptions", "")
-
-    // Set the environment variable through a command prefix
-    // to append to the existing value of the variable
-    val prefixEnv = conf.getOption("spark.executor.extraLibraryPath").map { p =>
-      Utils.libraryPathEnvPrefix(Seq(p))
-    }.getOrElse("")
-
-    environment.addVariables(
-      Environment.Variable.newBuilder()
-        .setName("SPARK_EXECUTOR_OPTS")
-        .setValue(extraJavaOpts)
-        .build())
-
-    sc.executorEnvs.foreach { case (key, value) =>
-      environment.addVariables(Environment.Variable.newBuilder()
-        .setName(key)
-        .setValue(value)
-        .build())
-    }
-    val command = CommandInfo.newBuilder()
-      .setEnvironment(environment)
-
-    val uri = conf.getOption("spark.executor.uri")
-      .orElse(Option(System.getenv("SPARK_EXECUTOR_URI")))
-
-    if (uri.isEmpty) {
-      val executorSparkHome = conf.getOption("spark.mesos.executor.home")
-        .orElse(sc.getSparkHome())
-        .getOrElse {
-          throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!")
-        }
-      val runScript = new File(executorSparkHome, "./bin/spark-class").getPath
-      command.setValue(
-        "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend"
-          .format(prefixEnv, runScript) +
-        s" --driver-url $driverURL" +
-        s" --executor-id $taskId" +
-        s" --hostname ${executorHostname(offer)}" +
-        s" --cores $numCores" +
-        s" --app-id $appId")
-    } else {
-      // Grab everything to the first '.'. We'll use that and '*' to
-      // glob the directory "correctly".
-      val basename = uri.get.split('/').last.split('.').head
-      command.setValue(
-        s"cd $basename*; $prefixEnv " +
-        "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" +
-        s" --driver-url $driverURL" +
-        s" --executor-id $taskId" +
-        s" --hostname ${executorHostname(offer)}" +
-        s" --cores $numCores" +
-        s" --app-id $appId")
-      command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get).setCache(useFetcherCache))
-    }
-
-    conf.getOption("spark.mesos.uris").foreach(setupUris(_, command, useFetcherCache))
-
-    command.build()
-  }
-
-  protected def driverURL: String = {
-    if (conf.contains("spark.testing")) {
-      "driverURL"
-    } else {
-      RpcEndpointAddress(
-        conf.get("spark.driver.host"),
-        conf.get("spark.driver.port").toInt,
-        CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
-    }
-  }
-
-  override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID) {}
-
-  override def registered(
-      d: org.apache.mesos.SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) {
-    appId = frameworkId.getValue
-    mesosExternalShuffleClient.foreach(_.init(appId))
-    markRegistered()
-  }
-
-  override def sufficientResourcesRegistered(): Boolean = {
-    totalCoresAcquired >= maxCores * minRegisteredRatio
-  }
-
-  override def disconnected(d: org.apache.mesos.SchedulerDriver) {}
-
-  override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {}
-
-  /**
-   * Method called by Mesos to offer resources on slaves. We respond by launching an executor,
-   * unless we've already launched more than we wanted to.
-   */
-  override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]) {
-    stateLock.synchronized {
-      if (stopCalled) {
-        logDebug("Ignoring offers during shutdown")
-        // Driver should simply return a stopped status on race
-        // condition between this.stop() and completing here
-        offers.asScala.map(_.getId).foreach(d.declineOffer)
-        return
-      }
-
-      logDebug(s"Received ${offers.size} resource offers.")
-
-      val (matchedOffers, unmatchedOffers) = offers.asScala.partition { offer =>
-        val offerAttributes = toAttributeMap(offer.getAttributesList)
-        matchesAttributeRequirements(slaveOfferConstraints, offerAttributes)
-      }
-
-      declineUnmatchedOffers(d, unmatchedOffers)
-      handleMatchedOffers(d, matchedOffers)
-    }
-  }
-
-  private def declineUnmatchedOffers(
-      d: org.apache.mesos.SchedulerDriver, offers: mutable.Buffer[Offer]): Unit = {
-    offers.foreach { offer =>
-      declineOffer(d, offer, Some("unmet constraints"),
-        Some(rejectOfferDurationForUnmetConstraints))
-    }
-  }
-
-  private def declineOffer(
-      d: org.apache.mesos.SchedulerDriver,
-      offer: Offer,
-      reason: Option[String] = None,
-      refuseSeconds: Option[Long] = None): Unit = {
-
-    val id = offer.getId.getValue
-    val offerAttributes = toAttributeMap(offer.getAttributesList)
-    val mem = getResource(offer.getResourcesList, "mem")
-    val cpus = getResource(offer.getResourcesList, "cpus")
-    val ports = getRangeResource(offer.getResourcesList, "ports")
-
-    logDebug(s"Declining offer: $id with attributes: $offerAttributes mem: $mem" +
-      s" cpu: $cpus port: $ports for $refuseSeconds seconds" +
-      reason.map(r => s" (reason: $r)").getOrElse(""))
-
-    refuseSeconds match {
-      case Some(seconds) =>
-        val filters = Filters.newBuilder().setRefuseSeconds(seconds).build()
-        d.declineOffer(offer.getId, filters)
-      case _ => d.declineOffer(offer.getId)
-    }
-  }
-
-  /**
-   * Launches executors on accepted offers, and declines unused offers. Executors are launched
-   * round-robin on offers.
-   *
-   * @param d SchedulerDriver
-   * @param offers Mesos offers that match attribute constraints
-   */
-  private def handleMatchedOffers(
-      d: org.apache.mesos.SchedulerDriver, offers: mutable.Buffer[Offer]): Unit = {
-    val tasks = buildMesosTasks(offers)
-    for (offer <- offers) {
-      val offerAttributes = toAttributeMap(offer.getAttributesList)
-      val offerMem = getResource(offer.getResourcesList, "mem")
-      val offerCpus = getResource(offer.getResourcesList, "cpus")
-      val offerPorts = getRangeResource(offer.getResourcesList, "ports")
-      val id = offer.getId.getValue
-
-      if (tasks.contains(offer.getId)) { // accept
-        val offerTasks = tasks(offer.getId)
-
-        logDebug(s"Accepting offer: $id with attributes: $offerAttributes " +
-          s"mem: $offerMem cpu: $offerCpus ports: $offerPorts." +
-          s"  Launching ${offerTasks.size} Mesos tasks.")
-
-        for (task <- offerTasks) {
-          val taskId = task.getTaskId
-          val mem = getResource(task.getResourcesList, "mem")
-          val cpus = getResource(task.getResourcesList, "cpus")
-          val ports = getRangeResource(task.getResourcesList, "ports").mkString(",")
-
-          logDebug(s"Launching Mesos task: ${taskId.getValue} with mem: $mem cpu: $cpus" +
-            s" ports: $ports")
-        }
-
-        d.launchTasks(
-          Collections.singleton(offer.getId),
-          offerTasks.asJava)
-      } else if (totalCoresAcquired >= maxCores) {
-        // Reject an offer for a configurable amount of time to avoid starving other frameworks
-        declineOffer(d, offer, Some("reached spark.cores.max"),
-          Some(rejectOfferDurationForReachedMaxCores))
-      } else {
-        declineOffer(d, offer)
-      }
-    }
-  }
-
-  /**
-   * Returns a map from OfferIDs to the tasks to launch on those offers.  In order to maximize
-   * per-task memory and IO, tasks are round-robin assigned to offers.
-   *
-   * @param offers Mesos offers that match attribute constraints
-   * @return A map from OfferID to a list of Mesos tasks to launch on that offer
-   */
-  private def buildMesosTasks(offers: mutable.Buffer[Offer]): Map[OfferID, List[MesosTaskInfo]] = {
-    // offerID -> tasks
-    val tasks = new mutable.HashMap[OfferID, List[MesosTaskInfo]].withDefaultValue(Nil)
-
-    // offerID -> resources
-    val remainingResources = mutable.Map(offers.map(offer =>
-      (offer.getId.getValue, offer.getResourcesList)): _*)
-
-    var launchTasks = true
-
-    // TODO(mgummelt): combine offers for a single slave
-    //
-    // round-robin create executors on the available offers
-    while (launchTasks) {
-      launchTasks = false
-
-      for (offer <- offers) {
-        val slaveId = offer.getSlaveId.getValue
-        val offerId = offer.getId.getValue
-        val resources = remainingResources(offerId)
-
-        if (canLaunchTask(slaveId, resources)) {
-          // Create a task
-          launchTasks = true
-          val taskId = newMesosTaskId()
-          val offerCPUs = getResource(resources, "cpus").toInt
-          val taskGPUs = Math.min(
-            Math.max(0, maxGpus - totalGpusAcquired), getResource(resources, "gpus").toInt)
-
-          val taskCPUs = executorCores(offerCPUs)
-          val taskMemory = executorMemory(sc)
-
-          slaves.getOrElseUpdate(slaveId, new Slave(offer.getHostname)).taskIDs.add(taskId)
-
-          val (resourcesLeft, resourcesToUse) =
-            partitionTaskResources(resources, taskCPUs, taskMemory, taskGPUs)
-
-          val taskBuilder = MesosTaskInfo.newBuilder()
-            .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build())
-            .setSlaveId(offer.getSlaveId)
-            .setCommand(createCommand(offer, taskCPUs + extraCoresPerExecutor, taskId))
-            .setName("Task " + taskId)
-          taskBuilder.addAllResources(resourcesToUse.asJava)
-          taskBuilder.setContainer(MesosSchedulerBackendUtil.containerInfo(sc.conf))
-
-          tasks(offer.getId) ::= taskBuilder.build()
-          remainingResources(offerId) = resourcesLeft.asJava
-          totalCoresAcquired += taskCPUs
-          coresByTaskId(taskId) = taskCPUs
-          if (taskGPUs > 0) {
-            totalGpusAcquired += taskGPUs
-            gpusByTaskId(taskId) = taskGPUs
-          }
-        }
-      }
-    }
-    tasks.toMap
-  }
-
-  /** Extracts task needed resources from a list of available resources. */
-  private def partitionTaskResources(
-      resources: JList[Resource],
-      taskCPUs: Int,
-      taskMemory: Int,
-      taskGPUs: Int)
-    : (List[Resource], List[Resource]) = {
-
-    // partition cpus & mem
-    val (afterCPUResources, cpuResourcesToUse) = partitionResources(resources, "cpus", taskCPUs)
-    val (afterMemResources, memResourcesToUse) =
-      partitionResources(afterCPUResources.asJava, "mem", taskMemory)
-    val (afterGPUResources, gpuResourcesToUse) =
-      partitionResources(afterMemResources.asJava, "gpus", taskGPUs)
-
-    // If user specifies port numbers in SparkConfig then consecutive tasks will not be launched
-    // on the same host. This essentially means one executor per host.
-    // TODO: handle network isolator case
-    val (nonPortResources, portResourcesToUse) =
-      partitionPortResources(nonZeroPortValuesFromConfig(sc.conf), afterGPUResources)
-
-    (nonPortResources,
-      cpuResourcesToUse ++ memResourcesToUse ++ portResourcesToUse ++ gpuResourcesToUse)
-  }
-
-  private def canLaunchTask(slaveId: String, resources: JList[Resource]): Boolean = {
-    val offerMem = getResource(resources, "mem")
-    val offerCPUs = getResource(resources, "cpus").toInt
-    val cpus = executorCores(offerCPUs)
-    val mem = executorMemory(sc)
-    val ports = getRangeResource(resources, "ports")
-    val meetsPortRequirements = checkPorts(sc.conf, ports)
-
-    cpus > 0 &&
-      cpus <= offerCPUs &&
-      cpus + totalCoresAcquired <= maxCores &&
-      mem <= offerMem &&
-      numExecutors() < executorLimit &&
-      slaves.get(slaveId).map(_.taskFailures).getOrElse(0) < MAX_SLAVE_FAILURES &&
-      meetsPortRequirements
-  }
-
-  private def executorCores(offerCPUs: Int): Int = {
-    sc.conf.getInt("spark.executor.cores",
-      math.min(offerCPUs, maxCores - totalCoresAcquired))
-  }
-
-  override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus) {
-    val taskId = status.getTaskId.getValue
-    val slaveId = status.getSlaveId.getValue
-    val state = mesosToTaskState(status.getState)
-
-    logInfo(s"Mesos task $taskId is now ${status.getState}")
-
-    stateLock.synchronized {
-      val slave = slaves(slaveId)
-
-      // If the shuffle service is enabled, have the driver register with each one of the
-      // shuffle services. This allows the shuffle services to clean up state associated with
-      // this application when the driver exits. There is currently not a great way to detect
-      // this through Mesos, since the shuffle services are set up independently.
-      if (state.equals(TaskState.RUNNING) &&
-          shuffleServiceEnabled &&
-          !slave.shuffleRegistered) {
-        assume(mesosExternalShuffleClient.isDefined,
-          "External shuffle client was not instantiated even though shuffle service is enabled.")
-        // TODO: Remove this and allow the MesosExternalShuffleService to detect
-        // framework termination when new Mesos Framework HTTP API is available.
-        val externalShufflePort = conf.getInt("spark.shuffle.service.port", 7337)
-
-        logDebug(s"Connecting to shuffle service on slave $slaveId, " +
-            s"host ${slave.hostname}, port $externalShufflePort for app ${conf.getAppId}")
-
-        mesosExternalShuffleClient.get
-          .registerDriverWithShuffleService(
-            slave.hostname,
-            externalShufflePort,
-            sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs",
-              s"${sc.conf.getTimeAsMs("spark.network.timeout", "120s")}ms"),
-            sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s"))
-        slave.shuffleRegistered = true
-      }
-
-      if (TaskState.isFinished(state)) {
-        // Remove the cores we have remembered for this task, if it's in the hashmap
-        for (cores <- coresByTaskId.get(taskId)) {
-          totalCoresAcquired -= cores
-          coresByTaskId -= taskId
-        }
-        // Also remove the gpus we have remembered for this task, if it's in the hashmap
-        for (gpus <- gpusByTaskId.get(taskId)) {
-          totalGpusAcquired -= gpus
-          gpusByTaskId -= taskId
-        }
-        // If it was a failure, mark the slave as failed for blacklisting purposes
-        if (TaskState.isFailed(state)) {
-          slave.taskFailures += 1
-
-          if (slave.taskFailures >= MAX_SLAVE_FAILURES) {
-            logInfo(s"Blacklisting Mesos slave $slaveId due to too many failures; " +
-                "is Spark installed on it?")
-          }
-        }
-        executorTerminated(d, slaveId, taskId, s"Executor finished with state $state")
-        // In case we'd rejected everything before but have now lost a node
-        d.reviveOffers()
-      }
-    }
-  }
-
-  override def error(d: org.apache.mesos.SchedulerDriver, message: String) {
-    logError(s"Mesos error: $message")
-    scheduler.error(message)
-  }
-
-  override def stop() {
-    // Make sure we're not launching tasks during shutdown
-    stateLock.synchronized {
-      if (stopCalled) {
-        logWarning("Stop called multiple times, ignoring")
-        return
-      }
-      stopCalled = true
-      super.stop()
-    }
-
-    // Wait for executors to report done, or else mesosDriver.stop() will forcefully kill them.
-    // See SPARK-12330
-    val startTime = System.nanoTime()
-
-    // slaveIdsWithExecutors has no memory barrier, so this is eventually consistent
-    while (numExecutors() > 0 &&
-      System.nanoTime() - startTime < shutdownTimeoutMS * 1000L * 1000L) {
-      Thread.sleep(100)
-    }
-
-    if (numExecutors() > 0) {
-      logWarning(s"Timed out waiting for ${numExecutors()} remaining executors "
-        + s"to terminate within $shutdownTimeoutMS ms. This may leave temporary files "
-        + "on the mesos nodes.")
-    }
-
-    // Close the mesos external shuffle client if used
-    mesosExternalShuffleClient.foreach(_.close())
-
-    if (mesosDriver != null) {
-      mesosDriver.stop()
-    }
-  }
-
-  override def frameworkMessage(
-      d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {}
-
-  /**
-   * Called when a slave is lost or a Mesos task finished. Updates local view on
-   * what tasks are running. It also notifies the driver that an executor was removed.
-   */
-  private def executorTerminated(
-      d: org.apache.mesos.SchedulerDriver,
-      slaveId: String,
-      taskId: String,
-      reason: String): Unit = {
-    stateLock.synchronized {
-      // Do not call removeExecutor() after this scheduler backend was stopped because
-      // removeExecutor() internally will send a message to the driver endpoint but
-      // the driver endpoint is not available now, otherwise an exception will be thrown.
-      if (!stopCalled) {
-        removeExecutor(taskId, SlaveLost(reason))
-      }
-      slaves(slaveId).taskIDs.remove(taskId)
-    }
-  }
-
-  override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID): Unit = {
-    logInfo(s"Mesos slave lost: ${slaveId.getValue}")
-  }
-
-  override def executorLost(
-      d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int): Unit = {
-    logInfo("Mesos executor lost: %s".format(e.getValue))
-  }
-
-  override def applicationId(): String =
-    Option(appId).getOrElse {
-      logWarning("Application ID is not initialized yet.")
-      super.applicationId
-    }
-
-  override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future.successful {
-    // We don't truly know if we can fulfill the full amount of executors
-    // since at coarse grain it depends on the amount of slaves available.
-    logInfo("Capping the total amount of executors to " + requestedTotal)
-    executorLimitOption = Some(requestedTotal)
-    true
-  }
-
-  override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future.successful {
-    if (mesosDriver == null) {
-      logWarning("Asked to kill executors before the Mesos driver was started.")
-      false
-    } else {
-      for (executorId <- executorIds) {
-        val taskId = TaskID.newBuilder().setValue(executorId).build()
-        mesosDriver.killTask(taskId)
-      }
-      // no need to adjust `executorLimitOption` since the AllocationManager already communicated
-      // the desired limit through a call to `doRequestTotalExecutors`.
-      // See [[o.a.s.scheduler.cluster.CoarseGrainedSchedulerBackend.killExecutors]]
-      true
-    }
-  }
-
-  private def numExecutors(): Int = {
-    slaves.values.map(_.taskIDs.size).sum
-  }
-
-  private def executorHostname(offer: Offer): String = {
-    if (sc.conf.getOption("spark.mesos.network.name").isDefined) {
-      // The agent's IP is not visible in a CNI container, so we bind to 0.0.0.0
-      "0.0.0.0"
-    } else {
-      offer.getHostname
-    }
-  }
-}
-
-private class Slave(val hostname: String) {
-  val taskIDs = new mutable.HashSet[String]()
-  var taskFailures = 0
-  var shuffleRegistered = false
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala
deleted file mode 100644
index 779ffb5..0000000
--- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala
+++ /dev/null
@@ -1,444 +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.mesos
-
-import java.io.File
-import java.util.{ArrayList => JArrayList, Collections, List => JList}
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.{HashMap, HashSet}
-
-import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, TaskInfo => MesosTaskInfo, _}
-import org.apache.mesos.protobuf.ByteString
-
-import org.apache.spark.{SparkContext, SparkException, TaskState}
-import org.apache.spark.executor.MesosExecutorBackend
-import org.apache.spark.scheduler._
-import org.apache.spark.scheduler.cluster.ExecutorInfo
-import org.apache.spark.util.Utils
-
-/**
- * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a
- * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks
- * from multiple apps can run on different cores) and in time (a core can switch ownership).
- */
-private[spark] class MesosFineGrainedSchedulerBackend(
-    scheduler: TaskSchedulerImpl,
-    sc: SparkContext,
-    master: String)
-  extends SchedulerBackend
-  with org.apache.mesos.Scheduler
-  with MesosSchedulerUtils {
-
-  // Stores the slave ids that has launched a Mesos executor.
-  val slaveIdToExecutorInfo = new HashMap[String, MesosExecutorInfo]
-  val taskIdToSlaveId = new HashMap[Long, String]
-
-  // An ExecutorInfo for our tasks
-  var execArgs: Array[Byte] = null
-
-  var classLoader: ClassLoader = null
-
-  // The listener bus to publish executor added/removed events.
-  val listenerBus = sc.listenerBus
-
-  private[mesos] val mesosExecutorCores = sc.conf.getDouble("spark.mesos.mesosExecutor.cores", 1)
-
-  // Offer constraints
-  private[this] val slaveOfferConstraints =
-    parseConstraintString(sc.conf.get("spark.mesos.constraints", ""))
-
-  // reject offers with mismatched constraints in seconds
-  private val rejectOfferDurationForUnmetConstraints =
-    getRejectOfferDurationForUnmetConstraints(sc)
-
-  @volatile var appId: String = _
-
-  override def start() {
-    classLoader = Thread.currentThread.getContextClassLoader
-    val driver = createSchedulerDriver(
-      master,
-      MesosFineGrainedSchedulerBackend.this,
-      sc.sparkUser,
-      sc.appName,
-      sc.conf,
-      sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.webUrl)),
-      Option.empty,
-      Option.empty,
-      sc.conf.getOption("spark.mesos.driver.frameworkId")
-    )
-
-    unsetFrameworkID(sc)
-    startScheduler(driver)
-  }
-
-  /**
-   * Creates a MesosExecutorInfo that is used to launch a Mesos executor.
-   * @param availableResources Available resources that is offered by Mesos
-   * @param execId The executor id to assign to this new executor.
-   * @return A tuple of the new mesos executor info and the remaining available resources.
-   */
-  def createExecutorInfo(
-      availableResources: JList[Resource],
-      execId: String): (MesosExecutorInfo, JList[Resource]) = {
-    val executorSparkHome = sc.conf.getOption("spark.mesos.executor.home")
-      .orElse(sc.getSparkHome()) // Fall back to driver Spark home for backward compatibility
-      .getOrElse {
-      throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!")
-    }
-    val environment = Environment.newBuilder()
-    sc.conf.getOption("spark.executor.extraClassPath").foreach { cp =>
-      environment.addVariables(
-        Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build())
-    }
-    val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions").getOrElse("")
-
-    val prefixEnv = sc.conf.getOption("spark.executor.extraLibraryPath").map { p =>
-      Utils.libraryPathEnvPrefix(Seq(p))
-    }.getOrElse("")
-
-    environment.addVariables(
-      Environment.Variable.newBuilder()
-        .setName("SPARK_EXECUTOR_OPTS")
-        .setValue(extraJavaOpts)
-        .build())
-    sc.executorEnvs.foreach { case (key, value) =>
-      environment.addVariables(Environment.Variable.newBuilder()
-        .setName(key)
-        .setValue(value)
-        .build())
-    }
-    val command = CommandInfo.newBuilder()
-      .setEnvironment(environment)
-    val uri = sc.conf.getOption("spark.executor.uri")
-      .orElse(Option(System.getenv("SPARK_EXECUTOR_URI")))
-
-    val executorBackendName = classOf[MesosExecutorBackend].getName
-    if (uri.isEmpty) {
-      val executorPath = new File(executorSparkHome, "/bin/spark-class").getPath
-      command.setValue(s"$prefixEnv $executorPath $executorBackendName")
-    } else {
-      // Grab everything to the first '.'. We'll use that and '*' to
-      // glob the directory "correctly".
-      val basename = uri.get.split('/').last.split('.').head
-      command.setValue(s"cd ${basename}*; $prefixEnv ./bin/spark-class $executorBackendName")
-      command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get))
-    }
-    val builder = MesosExecutorInfo.newBuilder()
-    val (resourcesAfterCpu, usedCpuResources) =
-      partitionResources(availableResources, "cpus", mesosExecutorCores)
-    val (resourcesAfterMem, usedMemResources) =
-      partitionResources(resourcesAfterCpu.asJava, "mem", executorMemory(sc))
-
-    builder.addAllResources(usedCpuResources.asJava)
-    builder.addAllResources(usedMemResources.asJava)
-
-    sc.conf.getOption("spark.mesos.uris").foreach(setupUris(_, command))
-
-    val executorInfo = builder
-      .setExecutorId(ExecutorID.newBuilder().setValue(execId).build())
-      .setCommand(command)
-      .setData(ByteString.copyFrom(createExecArg()))
-
-    executorInfo.setContainer(MesosSchedulerBackendUtil.containerInfo(sc.conf))
-    (executorInfo.build(), resourcesAfterMem.asJava)
-  }
-
-  /**
-   * Create and serialize the executor argument to pass to Mesos. Our executor arg is an array
-   * containing all the spark.* system properties in the form of (String, String) pairs.
-   */
-  private def createExecArg(): Array[Byte] = {
-    if (execArgs == null) {
-      val props = new HashMap[String, String]
-      for ((key, value) <- sc.conf.getAll) {
-        props(key) = value
-      }
-      // Serialize the map as an array of (String, String) pairs
-      execArgs = Utils.serialize(props.toArray)
-    }
-    execArgs
-  }
-
-  override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID) {}
-
-  override def registered(
-      d: org.apache.mesos.SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) {
-    inClassLoader() {
-      appId = frameworkId.getValue
-      logInfo("Registered as framework ID " + appId)
-      markRegistered()
-    }
-  }
-
-  private def inClassLoader()(fun: => Unit) = {
-    val oldClassLoader = Thread.currentThread.getContextClassLoader
-    Thread.currentThread.setContextClassLoader(classLoader)
-    try {
-      fun
-    } finally {
-      Thread.currentThread.setContextClassLoader(oldClassLoader)
-    }
-  }
-
-  override def disconnected(d: org.apache.mesos.SchedulerDriver) {}
-
-  override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {}
-
-  private def getTasksSummary(tasks: JArrayList[MesosTaskInfo]): String = {
-    val builder = new StringBuilder
-    tasks.asScala.foreach { t =>
-      builder.append("Task id: ").append(t.getTaskId.getValue).append("\n")
-        .append("Slave id: ").append(t.getSlaveId.getValue).append("\n")
-        .append("Task resources: ").append(t.getResourcesList).append("\n")
-        .append("Executor resources: ").append(t.getExecutor.getResourcesList)
-        .append("---------------------------------------------\n")
-    }
-    builder.toString()
-  }
-
-  /**
-   * Method called by Mesos to offer resources on slaves. We respond by asking our active task sets
-   * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that
-   * tasks are balanced across the cluster.
-   */
-  override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]) {
-    inClassLoader() {
-      // Fail first on offers with unmet constraints
-      val (offersMatchingConstraints, offersNotMatchingConstraints) =
-        offers.asScala.partition { o =>
-          val offerAttributes = toAttributeMap(o.getAttributesList)
-          val meetsConstraints =
-            matchesAttributeRequirements(slaveOfferConstraints, offerAttributes)
-
-          // add some debug messaging
-          if (!meetsConstraints) {
-            val id = o.getId.getValue
-            logDebug(s"Declining offer: $id with attributes: $offerAttributes")
-          }
-
-          meetsConstraints
-        }
-
-      // These offers do not meet constraints. We don't need to see them again.
-      // Decline the offer for a long period of time.
-      offersNotMatchingConstraints.foreach { o =>
-        d.declineOffer(o.getId, Filters.newBuilder()
-          .setRefuseSeconds(rejectOfferDurationForUnmetConstraints).build())
-      }
-
-      // Of the matching constraints, see which ones give us enough memory and cores
-      val (usableOffers, unUsableOffers) = offersMatchingConstraints.partition { o =>
-        val mem = getResource(o.getResourcesList, "mem")
-        val cpus = getResource(o.getResourcesList, "cpus")
-        val slaveId = o.getSlaveId.getValue
-        val offerAttributes = toAttributeMap(o.getAttributesList)
-
-        // check offers for
-        //  1. Memory requirements
-        //  2. CPU requirements - need at least 1 for executor, 1 for task
-        val meetsMemoryRequirements = mem >= executorMemory(sc)
-        val meetsCPURequirements = cpus >= (mesosExecutorCores + scheduler.CPUS_PER_TASK)
-        val meetsRequirements =
-          (meetsMemoryRequirements && meetsCPURequirements) ||
-          (slaveIdToExecutorInfo.contains(slaveId) && cpus >= scheduler.CPUS_PER_TASK)
-        val debugstr = if (meetsRequirements) "Accepting" else "Declining"
-        logDebug(s"$debugstr offer: ${o.getId.getValue} with attributes: "
-          + s"$offerAttributes mem: $mem cpu: $cpus")
-
-        meetsRequirements
-      }
-
-      // Decline offers we ruled out immediately
-      unUsableOffers.foreach(o => d.declineOffer(o.getId))
-
-      val workerOffers = usableOffers.map { o =>
-        val cpus = if (slaveIdToExecutorInfo.contains(o.getSlaveId.getValue)) {
-          getResource(o.getResourcesList, "cpus").toInt
-        } else {
-          // If the Mesos executor has not been started on this slave yet, set aside a few
-          // cores for the Mesos executor by offering fewer cores to the Spark executor
-          (getResource(o.getResourcesList, "cpus") - mesosExecutorCores).toInt
-        }
-        new WorkerOffer(
-          o.getSlaveId.getValue,
-          o.getHostname,
-          cpus)
-      }.toIndexedSeq
-
-      val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap
-      val slaveIdToWorkerOffer = workerOffers.map(o => o.executorId -> o).toMap
-      val slaveIdToResources = new HashMap[String, JList[Resource]]()
-      usableOffers.foreach { o =>
-        slaveIdToResources(o.getSlaveId.getValue) = o.getResourcesList
-      }
-
-      val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]]
-
-      val slavesIdsOfAcceptedOffers = HashSet[String]()
-
-      // Call into the TaskSchedulerImpl
-      val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty)
-      acceptedOffers
-        .foreach { offer =>
-          offer.foreach { taskDesc =>
-            val slaveId = taskDesc.executorId
-            slavesIdsOfAcceptedOffers += slaveId
-            taskIdToSlaveId(taskDesc.taskId) = slaveId
-            val (mesosTask, remainingResources) = createMesosTask(
-              taskDesc,
-              slaveIdToResources(slaveId),
-              slaveId)
-            mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo])
-              .add(mesosTask)
-            slaveIdToResources(slaveId) = remainingResources
-          }
-        }
-
-      // Reply to the offers
-      val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout?
-
-      mesosTasks.foreach { case (slaveId, tasks) =>
-        slaveIdToWorkerOffer.get(slaveId).foreach(o =>
-          listenerBus.post(SparkListenerExecutorAdded(System.currentTimeMillis(), slaveId,
-            // TODO: Add support for log urls for Mesos
-            new ExecutorInfo(o.host, o.cores, Map.empty)))
-        )
-        logTrace(s"Launching Mesos tasks on slave '$slaveId', tasks:\n${getTasksSummary(tasks)}")
-        d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters)
-      }
-
-      // Decline offers that weren't used
-      // NOTE: This logic assumes that we only get a single offer for each host in a given batch
-      for (o <- usableOffers if !slavesIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) {
-        d.declineOffer(o.getId)
-      }
-    }
-  }
-
-  /** Turn a Spark TaskDescription into a Mesos task and also resources unused by the task */
-  def createMesosTask(
-      task: TaskDescription,
-      resources: JList[Resource],
-      slaveId: String): (MesosTaskInfo, JList[Resource]) = {
-    val taskId = TaskID.newBuilder().setValue(task.taskId.toString).build()
-    val (executorInfo, remainingResources) = if (slaveIdToExecutorInfo.contains(slaveId)) {
-      (slaveIdToExecutorInfo(slaveId), resources)
-    } else {
-      createExecutorInfo(resources, slaveId)
-    }
-    slaveIdToExecutorInfo(slaveId) = executorInfo
-    val (finalResources, cpuResources) =
-      partitionResources(remainingResources, "cpus", scheduler.CPUS_PER_TASK)
-    val taskInfo = MesosTaskInfo.newBuilder()
-      .setTaskId(taskId)
-      .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build())
-      .setExecutor(executorInfo)
-      .setName(task.name)
-      .addAllResources(cpuResources.asJava)
-      .setData(MesosTaskLaunchData(task.serializedTask, task.attemptNumber).toByteString)
-      .build()
-    (taskInfo, finalResources.asJava)
-  }
-
-  override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus) {
-    inClassLoader() {
-      val tid = status.getTaskId.getValue.toLong
-      val state = mesosToTaskState(status.getState)
-      synchronized {
-        if (TaskState.isFailed(mesosToTaskState(status.getState))
-          && taskIdToSlaveId.contains(tid)) {
-          // We lost the executor on this slave, so remember that it's gone
-          removeExecutor(taskIdToSlaveId(tid), "Lost executor")
-        }
-        if (TaskState.isFinished(state)) {
-          taskIdToSlaveId.remove(tid)
-        }
-      }
-      scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer)
-    }
-  }
-
-  override def error(d: org.apache.mesos.SchedulerDriver, message: String) {
-    inClassLoader() {
-      logError("Mesos error: " + message)
-      markErr()
-      scheduler.error(message)
-    }
-  }
-
-  override def stop() {
-    if (mesosDriver != null) {
-      mesosDriver.stop()
-    }
-  }
-
-  override def reviveOffers() {
-    mesosDriver.reviveOffers()
-  }
-
-  override def frameworkMessage(
-      d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {}
-
-  /**
-   * Remove executor associated with slaveId in a thread safe manner.
-   */
-  private def removeExecutor(slaveId: String, reason: String) = {
-    synchronized {
-      listenerBus.post(SparkListenerExecutorRemoved(System.currentTimeMillis(), slaveId, reason))
-      slaveIdToExecutorInfo -= slaveId
-    }
-  }
-
-  private def recordSlaveLost(
-      d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) {
-    inClassLoader() {
-      logInfo("Mesos slave lost: " + slaveId.getValue)
-      removeExecutor(slaveId.getValue, reason.toString)
-      scheduler.executorLost(slaveId.getValue, reason)
-    }
-  }
-
-  override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID) {
-    recordSlaveLost(d, slaveId, SlaveLost())
-  }
-
-  override def executorLost(
-      d: org.apache.mesos.SchedulerDriver, executorId: ExecutorID, slaveId: SlaveID, status: Int) {
-    logInfo("Executor lost: %s, marking slave %s as lost".format(executorId.getValue,
-                                                                 slaveId.getValue))
-    recordSlaveLost(d, slaveId, ExecutorExited(status, exitCausedByApp = true))
-  }
-
-  override def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = {
-    mesosDriver.killTask(
-      TaskID.newBuilder()
-        .setValue(taskId.toString).build()
-    )
-  }
-
-  // TODO: query Mesos for number of cores
-  override def defaultParallelism(): Int = sc.conf.getInt("spark.default.parallelism", 8)
-
-  override def applicationId(): String =
-    Option(appId).getOrElse {
-      logWarning("Application ID is not initialized yet.")
-      super.applicationId
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala
deleted file mode 100644
index a2adb22..0000000
--- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala
+++ /dev/null
@@ -1,165 +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.mesos
-
-import org.apache.mesos.Protos.{ContainerInfo, Image, NetworkInfo, Volume}
-import org.apache.mesos.Protos.ContainerInfo.{DockerInfo, MesosInfo}
-
-import org.apache.spark.{SparkConf, SparkException}
-import org.apache.spark.internal.Logging
-
-/**
- * A collection of utility functions which can be used by both the
- * MesosSchedulerBackend and the [[MesosFineGrainedSchedulerBackend]].
- */
-private[mesos] object MesosSchedulerBackendUtil extends Logging {
-  /**
-   * Parse a comma-delimited list of volume specs, each of which
-   * takes the form [host-dir:]container-dir[:rw|:ro].
-   */
-  def parseVolumesSpec(volumes: String): List[Volume] = {
-    volumes.split(",").map(_.split(":")).flatMap { spec =>
-        val vol: Volume.Builder = Volume
-          .newBuilder()
-          .setMode(Volume.Mode.RW)
-        spec match {
-          case Array(container_path) =>
-            Some(vol.setContainerPath(container_path))
-          case Array(container_path, "rw") =>
-            Some(vol.setContainerPath(container_path))
-          case Array(container_path, "ro") =>
-            Some(vol.setContainerPath(container_path)
-              .setMode(Volume.Mode.RO))
-          case Array(host_path, container_path) =>
-            Some(vol.setContainerPath(container_path)
-              .setHostPath(host_path))
-          case Array(host_path, container_path, "rw") =>
-            Some(vol.setContainerPath(container_path)
-              .setHostPath(host_path))
-          case Array(host_path, container_path, "ro") =>
-            Some(vol.setContainerPath(container_path)
-              .setHostPath(host_path)
-              .setMode(Volume.Mode.RO))
-          case spec =>
-            logWarning(s"Unable to parse volume specs: $volumes. "
-              + "Expected form: \"[host-dir:]container-dir[:rw|:ro](, ...)\"")
-            None
-      }
-    }
-    .map { _.build() }
-    .toList
-  }
-
-  /**
-   * Parse a comma-delimited list of port mapping specs, each of which
-   * takes the form host_port:container_port[:udp|:tcp]
-   *
-   * Note:
-   * the docker form is [ip:]host_port:container_port, but the DockerInfo
-   * message has no field for 'ip', and instead has a 'protocol' field.
-   * Docker itself only appears to support TCP, so this alternative form
-   * anticipates the expansion of the docker form to allow for a protocol
-   * and leaves open the chance for mesos to begin to accept an 'ip' field
-   */
-  def parsePortMappingsSpec(portmaps: String): List[DockerInfo.PortMapping] = {
-    portmaps.split(",").map(_.split(":")).flatMap { spec: Array[String] =>
-      val portmap: DockerInfo.PortMapping.Builder = DockerInfo.PortMapping
-        .newBuilder()
-        .setProtocol("tcp")
-      spec match {
-        case Array(host_port, container_port) =>
-          Some(portmap.setHostPort(host_port.toInt)
-            .setContainerPort(container_port.toInt))
-        case Array(host_port, container_port, protocol) =>
-          Some(portmap.setHostPort(host_port.toInt)
-            .setContainerPort(container_port.toInt)
-            .setProtocol(protocol))
-        case spec =>
-          logWarning(s"Unable to parse port mapping specs: $portmaps. "
-            + "Expected form: \"host_port:container_port[:udp|:tcp](, ...)\"")
-          None
-      }
-    }
-    .map { _.build() }
-    .toList
-  }
-
-  def containerInfo(conf: SparkConf): ContainerInfo = {
-    val containerType = if (conf.contains("spark.mesos.executor.docker.image") &&
-      conf.get("spark.mesos.containerizer", "docker") == "docker") {
-      ContainerInfo.Type.DOCKER
-    } else {
-      ContainerInfo.Type.MESOS
-    }
-
-    val containerInfo = ContainerInfo.newBuilder()
-      .setType(containerType)
-
-    conf.getOption("spark.mesos.executor.docker.image").map { image =>
-      val forcePullImage = conf
-        .getOption("spark.mesos.executor.docker.forcePullImage")
-        .exists(_.equals("true"))
-
-      val portMaps = conf
-        .getOption("spark.mesos.executor.docker.portmaps")
-        .map(parsePortMappingsSpec)
-        .getOrElse(List.empty)
-
-      if (containerType == ContainerInfo.Type.DOCKER) {
-        containerInfo.setDocker(dockerInfo(image, forcePullImage, portMaps))
-      } else {
-        containerInfo.setMesos(mesosInfo(image, forcePullImage))
-      }
-
-      val volumes = conf
-        .getOption("spark.mesos.executor.docker.volumes")
-        .map(parseVolumesSpec)
-
-      volumes.foreach(_.foreach(containerInfo.addVolumes(_)))
-    }
-
-    conf.getOption("spark.mesos.network.name").map { name =>
-      val info = NetworkInfo.newBuilder().setName(name).build()
-      containerInfo.addNetworkInfos(info)
-    }
-
-    containerInfo.build()
-  }
-
-  private def dockerInfo(
-      image: String,
-      forcePullImage: Boolean,
-      portMaps: List[ContainerInfo.DockerInfo.PortMapping]): DockerInfo = {
-    val dockerBuilder = ContainerInfo.DockerInfo.newBuilder()
-      .setImage(image)
-      .setForcePullImage(forcePullImage)
-    portMaps.foreach(dockerBuilder.addPortMappings(_))
-
-    dockerBuilder.build
-  }
-
-  private def mesosInfo(image: String, forcePullImage: Boolean): MesosInfo = {
-    val imageProto = Image.newBuilder()
-      .setType(Image.Type.DOCKER)
-      .setDocker(Image.Docker.newBuilder().setName(image))
-      .setCached(!forcePullImage)
-    ContainerInfo.MesosInfo.newBuilder()
-      .setImage(imageProto)
-      .build
-  }
-}


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


[15/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala
----------------------------------------------------------------------
diff --git a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala
deleted file mode 100644
index ec47ab1..0000000
--- a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala
+++ /dev/null
@@ -1,256 +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.mesos
-
-import scala.collection.JavaConverters._
-import scala.language.reflectiveCalls
-
-import org.apache.mesos.Protos.{Resource, Value}
-import org.mockito.Mockito._
-import org.scalatest._
-import org.scalatest.mock.MockitoSugar
-
-import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
-import org.apache.spark.internal.config._
-
-class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoSugar {
-
-  // scalastyle:off structural.type
-  // this is the documented way of generating fixtures in scalatest
-  def fixture: Object {val sc: SparkContext; val sparkConf: SparkConf} = new {
-    val sparkConf = new SparkConf
-    val sc = mock[SparkContext]
-    when(sc.conf).thenReturn(sparkConf)
-  }
-
-  private def createTestPortResource(range: (Long, Long), role: Option[String] = None): Resource = {
-    val rangeValue = Value.Range.newBuilder()
-    rangeValue.setBegin(range._1)
-    rangeValue.setEnd(range._2)
-    val builder = Resource.newBuilder()
-      .setName("ports")
-      .setType(Value.Type.RANGES)
-      .setRanges(Value.Ranges.newBuilder().addRange(rangeValue))
-
-    role.foreach { r => builder.setRole(r) }
-    builder.build()
-  }
-
-  private def rangesResourcesToTuple(resources: List[Resource]): List[(Long, Long)] = {
-    resources.flatMap{resource => resource.getRanges.getRangeList
-      .asScala.map(range => (range.getBegin, range.getEnd))}
-  }
-
-  def arePortsEqual(array1: Array[(Long, Long)], array2: Array[(Long, Long)])
-    : Boolean = {
-    array1.sortBy(identity).deep == array2.sortBy(identity).deep
-  }
-
-  def arePortsEqual(array1: Array[Long], array2: Array[Long])
-    : Boolean = {
-    array1.sortBy(identity).deep == array2.sortBy(identity).deep
-  }
-
-  def getRangesFromResources(resources: List[Resource]): List[(Long, Long)] = {
-    resources.flatMap{ resource =>
-      resource.getRanges.getRangeList.asScala.toList.map{
-        range => (range.getBegin, range.getEnd)}}
-  }
-
-  val utils = new MesosSchedulerUtils { }
-  // scalastyle:on structural.type
-
-  test("use at-least minimum overhead") {
-    val f = fixture
-    when(f.sc.executorMemory).thenReturn(512)
-    utils.executorMemory(f.sc) shouldBe 896
-  }
-
-  test("use overhead if it is greater than minimum value") {
-    val f = fixture
-    when(f.sc.executorMemory).thenReturn(4096)
-    utils.executorMemory(f.sc) shouldBe 4505
-  }
-
-  test("use spark.mesos.executor.memoryOverhead (if set)") {
-    val f = fixture
-    when(f.sc.executorMemory).thenReturn(1024)
-    f.sparkConf.set("spark.mesos.executor.memoryOverhead", "512")
-    utils.executorMemory(f.sc) shouldBe 1536
-  }
-
-  test("parse a non-empty constraint string correctly") {
-    val expectedMap = Map(
-      "os" -> Set("centos7"),
-      "zone" -> Set("us-east-1a", "us-east-1b")
-    )
-    utils.parseConstraintString("os:centos7;zone:us-east-1a,us-east-1b") should be (expectedMap)
-  }
-
-  test("parse an empty constraint string correctly") {
-    utils.parseConstraintString("") shouldBe Map()
-  }
-
-  test("throw an exception when the input is malformed") {
-    an[IllegalArgumentException] should be thrownBy
-      utils.parseConstraintString("os;zone:us-east")
-  }
-
-  test("empty values for attributes' constraints matches all values") {
-    val constraintsStr = "os:"
-    val parsedConstraints = utils.parseConstraintString(constraintsStr)
-
-    parsedConstraints shouldBe Map("os" -> Set())
-
-    val zoneSet = Value.Set.newBuilder().addItem("us-east-1a").addItem("us-east-1b").build()
-    val noOsOffer = Map("zone" -> zoneSet)
-    val centosOffer = Map("os" -> Value.Text.newBuilder().setValue("centos").build())
-    val ubuntuOffer = Map("os" -> Value.Text.newBuilder().setValue("ubuntu").build())
-
-    utils.matchesAttributeRequirements(parsedConstraints, noOsOffer) shouldBe false
-    utils.matchesAttributeRequirements(parsedConstraints, centosOffer) shouldBe true
-    utils.matchesAttributeRequirements(parsedConstraints, ubuntuOffer) shouldBe true
-  }
-
-  test("subset match is performed for set attributes") {
-    val supersetConstraint = Map(
-      "os" -> Value.Text.newBuilder().setValue("ubuntu").build(),
-      "zone" -> Value.Set.newBuilder()
-        .addItem("us-east-1a")
-        .addItem("us-east-1b")
-        .addItem("us-east-1c")
-        .build())
-
-    val zoneConstraintStr = "os:;zone:us-east-1a,us-east-1c"
-    val parsedConstraints = utils.parseConstraintString(zoneConstraintStr)
-
-    utils.matchesAttributeRequirements(parsedConstraints, supersetConstraint) shouldBe true
-  }
-
-  test("less than equal match is performed on scalar attributes") {
-    val offerAttribs = Map("gpus" -> Value.Scalar.newBuilder().setValue(3).build())
-
-    val ltConstraint = utils.parseConstraintString("gpus:2")
-    val eqConstraint = utils.parseConstraintString("gpus:3")
-    val gtConstraint = utils.parseConstraintString("gpus:4")
-
-    utils.matchesAttributeRequirements(ltConstraint, offerAttribs) shouldBe true
-    utils.matchesAttributeRequirements(eqConstraint, offerAttribs) shouldBe true
-    utils.matchesAttributeRequirements(gtConstraint, offerAttribs) shouldBe false
-  }
-
-  test("contains match is performed for range attributes") {
-    val offerAttribs = Map("ports" -> Value.Range.newBuilder().setBegin(7000).setEnd(8000).build())
-    val ltConstraint = utils.parseConstraintString("ports:6000")
-    val eqConstraint = utils.parseConstraintString("ports:7500")
-    val gtConstraint = utils.parseConstraintString("ports:8002")
-    val multiConstraint = utils.parseConstraintString("ports:5000,7500,8300")
-
-    utils.matchesAttributeRequirements(ltConstraint, offerAttribs) shouldBe false
-    utils.matchesAttributeRequirements(eqConstraint, offerAttribs) shouldBe true
-    utils.matchesAttributeRequirements(gtConstraint, offerAttribs) shouldBe false
-    utils.matchesAttributeRequirements(multiConstraint, offerAttribs) shouldBe true
-  }
-
-  test("equality match is performed for text attributes") {
-    val offerAttribs = Map("os" -> Value.Text.newBuilder().setValue("centos7").build())
-
-    val trueConstraint = utils.parseConstraintString("os:centos7")
-    val falseConstraint = utils.parseConstraintString("os:ubuntu")
-
-    utils.matchesAttributeRequirements(trueConstraint, offerAttribs) shouldBe true
-    utils.matchesAttributeRequirements(falseConstraint, offerAttribs) shouldBe false
-  }
-
-  test("Port reservation is done correctly with user specified ports only") {
-    val conf = new SparkConf()
-    conf.set("spark.executor.port", "3000" )
-    conf.set(BLOCK_MANAGER_PORT, 4000)
-    val portResource = createTestPortResource((3000, 5000), Some("my_role"))
-
-    val (resourcesLeft, resourcesToBeUsed) = utils
-      .partitionPortResources(List(3000, 4000), List(portResource))
-    resourcesToBeUsed.length shouldBe 2
-
-    val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1}.toArray
-
-    portsToUse.length shouldBe 2
-    arePortsEqual(portsToUse, Array(3000L, 4000L)) shouldBe true
-
-    val portRangesToBeUsed = rangesResourcesToTuple(resourcesToBeUsed)
-
-    val expectedUSed = Array((3000L, 3000L), (4000L, 4000L))
-
-    arePortsEqual(portRangesToBeUsed.toArray, expectedUSed) shouldBe true
-  }
-
-  test("Port reservation is done correctly with some user specified ports (spark.executor.port)") {
-    val conf = new SparkConf()
-    conf.set("spark.executor.port", "3100" )
-    val portResource = createTestPortResource((3000, 5000), Some("my_role"))
-
-    val (resourcesLeft, resourcesToBeUsed) = utils
-      .partitionPortResources(List(3100), List(portResource))
-
-    val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1}
-
-    portsToUse.length shouldBe 1
-    portsToUse.contains(3100) shouldBe true
-  }
-
-  test("Port reservation is done correctly with all random ports") {
-    val conf = new SparkConf()
-    val portResource = createTestPortResource((3000L, 5000L), Some("my_role"))
-
-    val (resourcesLeft, resourcesToBeUsed) = utils
-      .partitionPortResources(List(), List(portResource))
-    val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1}
-
-    portsToUse.isEmpty shouldBe true
-  }
-
-  test("Port reservation is done correctly with user specified ports only - multiple ranges") {
-    val conf = new SparkConf()
-    conf.set("spark.executor.port", "2100" )
-    conf.set("spark.blockManager.port", "4000")
-    val portResourceList = List(createTestPortResource((3000, 5000), Some("my_role")),
-      createTestPortResource((2000, 2500), Some("other_role")))
-    val (resourcesLeft, resourcesToBeUsed) = utils
-      .partitionPortResources(List(2100, 4000), portResourceList)
-    val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1}
-
-    portsToUse.length shouldBe 2
-    val portsRangesLeft = rangesResourcesToTuple(resourcesLeft)
-    val portRangesToBeUsed = rangesResourcesToTuple(resourcesToBeUsed)
-
-    val expectedUsed = Array((2100L, 2100L), (4000L, 4000L))
-
-    arePortsEqual(portsToUse.toArray, Array(2100L, 4000L)) shouldBe true
-    arePortsEqual(portRangesToBeUsed.toArray, expectedUsed) shouldBe true
-  }
-
-  test("Port reservation is done correctly with all random ports - multiple ranges") {
-    val conf = new SparkConf()
-    val portResourceList = List(createTestPortResource((3000, 5000), Some("my_role")),
-      createTestPortResource((2000, 2500), Some("other_role")))
-    val (resourcesLeft, resourcesToBeUsed) = utils
-      .partitionPortResources(List(), portResourceList)
-    val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1}
-    portsToUse.isEmpty shouldBe true
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala
----------------------------------------------------------------------
diff --git a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala
deleted file mode 100644
index 5a81bb3..0000000
--- a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala
+++ /dev/null
@@ -1,36 +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.mesos
-
-import java.nio.ByteBuffer
-
-import org.apache.spark.SparkFunSuite
-
-class MesosTaskLaunchDataSuite extends SparkFunSuite {
-  test("serialize and deserialize data must be same") {
-    val serializedTask = ByteBuffer.allocate(40)
-    (Range(100, 110).map(serializedTask.putInt(_)))
-    serializedTask.rewind
-    val attemptNumber = 100
-    val byteString = MesosTaskLaunchData(serializedTask, attemptNumber).toByteString
-    serializedTask.rewind
-    val mesosTaskLaunchData = MesosTaskLaunchData.fromByteString(byteString)
-    assert(mesosTaskLaunchData.attemptNumber == attemptNumber)
-    assert(mesosTaskLaunchData.serializedTask.equals(serializedTask))
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala
----------------------------------------------------------------------
diff --git a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala
deleted file mode 100644
index 7ebb294..0000000
--- a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala
+++ /dev/null
@@ -1,91 +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.mesos
-
-import java.util.Collections
-
-import scala.collection.JavaConverters._
-
-import org.apache.mesos.Protos._
-import org.apache.mesos.Protos.Value.{Range => MesosRange, Ranges, Scalar}
-import org.apache.mesos.SchedulerDriver
-import org.mockito.{ArgumentCaptor, Matchers}
-import org.mockito.Mockito._
-
-object Utils {
-  def createOffer(
-      offerId: String,
-      slaveId: String,
-      mem: Int,
-      cpus: Int,
-      ports: Option[(Long, Long)] = None,
-      gpus: Int = 0): Offer = {
-    val builder = Offer.newBuilder()
-    builder.addResourcesBuilder()
-      .setName("mem")
-      .setType(Value.Type.SCALAR)
-      .setScalar(Scalar.newBuilder().setValue(mem))
-    builder.addResourcesBuilder()
-      .setName("cpus")
-      .setType(Value.Type.SCALAR)
-      .setScalar(Scalar.newBuilder().setValue(cpus))
-    ports.foreach { resourcePorts =>
-      builder.addResourcesBuilder()
-        .setName("ports")
-        .setType(Value.Type.RANGES)
-        .setRanges(Ranges.newBuilder().addRange(MesosRange.newBuilder()
-          .setBegin(resourcePorts._1).setEnd(resourcePorts._2).build()))
-    }
-    if (gpus > 0) {
-      builder.addResourcesBuilder()
-        .setName("gpus")
-        .setType(Value.Type.SCALAR)
-        .setScalar(Scalar.newBuilder().setValue(gpus))
-    }
-    builder.setId(createOfferId(offerId))
-      .setFrameworkId(FrameworkID.newBuilder()
-        .setValue("f1"))
-      .setSlaveId(SlaveID.newBuilder().setValue(slaveId))
-      .setHostname(s"host${slaveId}")
-      .build()
-  }
-
-  def verifyTaskLaunched(driver: SchedulerDriver, offerId: String): List[TaskInfo] = {
-    val captor = ArgumentCaptor.forClass(classOf[java.util.Collection[TaskInfo]])
-    verify(driver, times(1)).launchTasks(
-      Matchers.eq(Collections.singleton(createOfferId(offerId))),
-      captor.capture())
-    captor.getValue.asScala.toList
-  }
-
-  def createOfferId(offerId: String): OfferID = {
-    OfferID.newBuilder().setValue(offerId).build()
-  }
-
-  def createSlaveId(slaveId: String): SlaveID = {
-    SlaveID.newBuilder().setValue(slaveId).build()
-  }
-
-  def createExecutorId(executorId: String): ExecutorID = {
-    ExecutorID.newBuilder().setValue(executorId).build()
-  }
-
-  def createTaskId(taskId: String): TaskID = {
-    TaskID.newBuilder().setValue(taskId).build()
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2f61d33..5e7dc15 100644
--- a/pom.xml
+++ b/pom.xml
@@ -2580,7 +2580,7 @@
     <profile>
       <id>yarn</id>
       <modules>
-        <module>yarn</module>
+        <module>resource-managers/yarn</module>
         <module>common/network-yarn</module>
       </modules>
     </profile>
@@ -2588,7 +2588,7 @@
     <profile>
       <id>mesos</id>
       <modules>
-        <module>mesos</module>
+        <module>resource-managers/mesos</module>
       </modules>
     </profile>
 

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml
new file mode 100644
index 0000000..c0a8f9a
--- /dev/null
+++ b/resource-managers/mesos/pom.xml
@@ -0,0 +1,109 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent_2.11</artifactId>
+    <version>2.2.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>spark-mesos_2.11</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project Mesos</name>
+  <properties>
+    <sbt.project.name>mesos</sbt.project.name>
+    <mesos.version>1.0.0</mesos.version>
+    <mesos.classifier>shaded-protobuf</mesos.classifier>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.mesos</groupId>
+      <artifactId>mesos</artifactId>
+      <version>${mesos.version}</version>
+      <classifier>${mesos.classifier}</classifier>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.protobuf</groupId>
+          <artifactId>protobuf-java</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- Explicitly depend on shaded dependencies from the parent, since shaded deps aren't transitive -->
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-plus</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-http</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlet</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlets</artifactId>
+    </dependency>
+    <!-- End of shaded deps. -->
+
+  </dependencies>
+
+
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
new file mode 100644
index 0000000..12b6d5b
--- /dev/null
+++ b/resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
@@ -0,0 +1 @@
+org.apache.spark.scheduler.cluster.mesos.MesosClusterManager

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala
new file mode 100644
index 0000000..792ade8
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala
@@ -0,0 +1,119 @@
+/*
+ * 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.mesos
+
+import java.util.concurrent.CountDownLatch
+
+import org.apache.spark.{SecurityManager, SparkConf}
+import org.apache.spark.deploy.mesos.config._
+import org.apache.spark.deploy.mesos.ui.MesosClusterUI
+import org.apache.spark.deploy.rest.mesos.MesosRestServer
+import org.apache.spark.internal.Logging
+import org.apache.spark.scheduler.cluster.mesos._
+import org.apache.spark.util.{CommandLineUtils, ShutdownHookManager, Utils}
+
+/*
+ * A dispatcher that is responsible for managing and launching drivers, and is intended to be
+ * used for Mesos cluster mode. The dispatcher is a long-running process started by the user in
+ * the cluster independently of Spark applications.
+ * It contains a [[MesosRestServer]] that listens for requests to submit drivers and a
+ * [[MesosClusterScheduler]] that processes these requests by negotiating with the Mesos master
+ * for resources.
+ *
+ * A typical new driver lifecycle is the following:
+ * - Driver submitted via spark-submit talking to the [[MesosRestServer]]
+ * - [[MesosRestServer]] queues the driver request to [[MesosClusterScheduler]]
+ * - [[MesosClusterScheduler]] gets resource offers and launches the drivers that are in queue
+ *
+ * This dispatcher supports both Mesos fine-grain or coarse-grain mode as the mode is configurable
+ * per driver launched.
+ * This class is needed since Mesos doesn't manage frameworks, so the dispatcher acts as
+ * a daemon to launch drivers as Mesos frameworks upon request. The dispatcher is also started and
+ * stopped by sbin/start-mesos-dispatcher and sbin/stop-mesos-dispatcher respectively.
+ */
+private[mesos] class MesosClusterDispatcher(
+    args: MesosClusterDispatcherArguments,
+    conf: SparkConf)
+  extends Logging {
+
+  private val publicAddress = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(args.host)
+  private val recoveryMode = conf.get(RECOVERY_MODE).toUpperCase()
+  logInfo("Recovery mode in Mesos dispatcher set to: " + recoveryMode)
+
+  private val engineFactory = recoveryMode match {
+    case "NONE" => new BlackHoleMesosClusterPersistenceEngineFactory
+    case "ZOOKEEPER" => new ZookeeperMesosClusterPersistenceEngineFactory(conf)
+    case _ => throw new IllegalArgumentException("Unsupported recovery mode: " + recoveryMode)
+  }
+
+  private val scheduler = new MesosClusterScheduler(engineFactory, conf)
+
+  private val server = new MesosRestServer(args.host, args.port, conf, scheduler)
+  private val webUi = new MesosClusterUI(
+    new SecurityManager(conf),
+    args.webUiPort,
+    conf,
+    publicAddress,
+    scheduler)
+
+  private val shutdownLatch = new CountDownLatch(1)
+
+  def start(): Unit = {
+    webUi.bind()
+    scheduler.frameworkUrl = conf.get(DISPATCHER_WEBUI_URL).getOrElse(webUi.activeWebUiUrl)
+    scheduler.start()
+    server.start()
+  }
+
+  def awaitShutdown(): Unit = {
+    shutdownLatch.await()
+  }
+
+  def stop(): Unit = {
+    webUi.stop()
+    server.stop()
+    scheduler.stop()
+    shutdownLatch.countDown()
+  }
+}
+
+private[mesos] object MesosClusterDispatcher
+  extends Logging
+  with CommandLineUtils {
+
+  override def main(args: Array[String]) {
+    Utils.initDaemon(log)
+    val conf = new SparkConf
+    val dispatcherArgs = new MesosClusterDispatcherArguments(args, conf)
+    conf.setMaster(dispatcherArgs.masterUrl)
+    conf.setAppName(dispatcherArgs.name)
+    dispatcherArgs.zookeeperUrl.foreach { z =>
+      conf.set(RECOVERY_MODE, "ZOOKEEPER")
+      conf.set(ZOOKEEPER_URL, z)
+    }
+    val dispatcher = new MesosClusterDispatcher(dispatcherArgs, conf)
+    dispatcher.start()
+    logDebug("Adding shutdown hook") // force eager creation of logger
+    ShutdownHookManager.addShutdownHook { () =>
+      logInfo("Shutdown hook is shutting down dispatcher")
+      dispatcher.stop()
+      dispatcher.awaitShutdown()
+    }
+    dispatcher.awaitShutdown()
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
new file mode 100644
index 0000000..ef08502
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
@@ -0,0 +1,149 @@
+/*
+ * 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.mesos
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+
+import org.apache.spark.util.{IntParam, Utils}
+import org.apache.spark.SparkConf
+
+private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf: SparkConf) {
+  var host: String = Utils.localHostName()
+  var port: Int = 7077
+  var name: String = "Spark Cluster"
+  var webUiPort: Int = 8081
+  var verbose: Boolean = false
+  var masterUrl: String = _
+  var zookeeperUrl: Option[String] = None
+  var propertiesFile: String = _
+  val confProperties: mutable.HashMap[String, String] =
+    new mutable.HashMap[String, String]()
+
+  parse(args.toList)
+
+  // scalastyle:on println
+  propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile)
+  Utils.updateSparkConfigFromProperties(conf, confProperties)
+
+  // scalastyle:off println
+  if (verbose) {
+    MesosClusterDispatcher.printStream.println(s"Using host: $host")
+    MesosClusterDispatcher.printStream.println(s"Using port: $port")
+    MesosClusterDispatcher.printStream.println(s"Using webUiPort: $webUiPort")
+    MesosClusterDispatcher.printStream.println(s"Framework Name: $name")
+
+    Option(propertiesFile).foreach { file =>
+      MesosClusterDispatcher.printStream.println(s"Using properties file: $file")
+    }
+
+    MesosClusterDispatcher.printStream.println(s"Spark Config properties set:")
+    conf.getAll.foreach(println)
+  }
+
+  @tailrec
+  private def parse(args: List[String]): Unit = args match {
+    case ("--host" | "-h") :: value :: tail =>
+      Utils.checkHost(value, "Please use hostname " + value)
+      host = value
+      parse(tail)
+
+    case ("--port" | "-p") :: IntParam(value) :: tail =>
+      port = value
+      parse(tail)
+
+    case ("--webui-port") :: IntParam(value) :: tail =>
+      webUiPort = value
+      parse(tail)
+
+    case ("--zk" | "-z") :: value :: tail =>
+      zookeeperUrl = Some(value)
+      parse(tail)
+
+    case ("--master" | "-m") :: value :: tail =>
+      if (!value.startsWith("mesos://")) {
+        // scalastyle:off println
+        MesosClusterDispatcher.printStream
+          .println("Cluster dispatcher only supports mesos (uri begins with mesos://)")
+        // scalastyle:on println
+        MesosClusterDispatcher.exitFn(1)
+      }
+      masterUrl = value.stripPrefix("mesos://")
+      parse(tail)
+
+    case ("--name") :: value :: tail =>
+      name = value
+      parse(tail)
+
+    case ("--properties-file") :: value :: tail =>
+      propertiesFile = value
+      parse(tail)
+
+    case ("--conf") :: value :: tail =>
+      val pair = MesosClusterDispatcher.
+        parseSparkConfProperty(value)
+        confProperties(pair._1) = pair._2
+      parse(tail)
+
+    case ("--help") :: tail =>
+        printUsageAndExit(0)
+
+    case ("--verbose") :: tail =>
+      verbose = true
+      parse(tail)
+
+    case Nil =>
+      if (Option(masterUrl).isEmpty) {
+        // scalastyle:off println
+        MesosClusterDispatcher.printStream.println("--master is required")
+        // scalastyle:on println
+        printUsageAndExit(1)
+      }
+
+    case value =>
+      // scalastyle:off println
+      MesosClusterDispatcher.printStream.println(s"Unrecognized option: '${value.head}'")
+      // scalastyle:on println
+      printUsageAndExit(1)
+  }
+
+  private def printUsageAndExit(exitCode: Int): Unit = {
+    val outStream = MesosClusterDispatcher.printStream
+
+    // scalastyle:off println
+    outStream.println(
+      "Usage: MesosClusterDispatcher [options]\n" +
+        "\n" +
+        "Options:\n" +
+        "  -h HOST, --host HOST    Hostname to listen on\n" +
+        "  --help                  Show this help message and exit.\n" +
+        "  --verbose,              Print additional debug output.\n" +
+        "  -p PORT, --port PORT    Port to listen on (default: 7077)\n" +
+        "  --webui-port WEBUI_PORT WebUI Port to listen on (default: 8081)\n" +
+        "  --name NAME             Framework name to show in Mesos UI\n" +
+        "  -m --master MASTER      URI for connecting to Mesos master\n" +
+        "  -z --zk ZOOKEEPER       Comma delimited URLs for connecting to \n" +
+        "                          Zookeeper for persistence\n" +
+        "  --properties-file FILE  Path to a custom Spark properties file.\n" +
+        "                          Default is conf/spark-defaults.conf \n" +
+        "  --conf PROP=VALUE       Arbitrary Spark configuration property.\n" +
+        "                          Takes precedence over defined properties in properties-file.")
+    // scalastyle:on println
+    MesosClusterDispatcher.exitFn(exitCode)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala
new file mode 100644
index 0000000..d4c7022
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala
@@ -0,0 +1,70 @@
+/*
+ * 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.mesos
+
+import java.util.Date
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.Command
+import org.apache.spark.scheduler.cluster.mesos.MesosClusterRetryState
+
+/**
+ * Describes a Spark driver that is submitted from the
+ * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]], to be launched by
+ * [[org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler]].
+ * @param jarUrl URL to the application jar
+ * @param mem Amount of memory for the driver
+ * @param cores Number of cores for the driver
+ * @param supervise Supervise the driver for long running app
+ * @param command The command to launch the driver.
+ * @param schedulerProperties Extra properties to pass the Mesos scheduler
+ */
+private[spark] class MesosDriverDescription(
+    val name: String,
+    val jarUrl: String,
+    val mem: Int,
+    val cores: Double,
+    val supervise: Boolean,
+    val command: Command,
+    schedulerProperties: Map[String, String],
+    val submissionId: String,
+    val submissionDate: Date,
+    val retryState: Option[MesosClusterRetryState] = None)
+  extends Serializable {
+
+  val conf = new SparkConf(false)
+  schedulerProperties.foreach {case (k, v) => conf.set(k, v)}
+
+  def copy(
+      name: String = name,
+      jarUrl: String = jarUrl,
+      mem: Int = mem,
+      cores: Double = cores,
+      supervise: Boolean = supervise,
+      command: Command = command,
+      schedulerProperties: SparkConf = conf,
+      submissionId: String = submissionId,
+      submissionDate: Date = submissionDate,
+      retryState: Option[MesosClusterRetryState] = retryState): MesosDriverDescription = {
+
+    new MesosDriverDescription(name, jarUrl, mem, cores, supervise, command, conf.getAll.toMap,
+      submissionId, submissionDate, retryState)
+  }
+
+  override def toString: String = s"MesosDriverDescription (${command.mainClass})"
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala
new file mode 100644
index 0000000..859aa83
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala
@@ -0,0 +1,131 @@
+/*
+ * 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.mesos
+
+import java.nio.ByteBuffer
+import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{SecurityManager, SparkConf}
+import org.apache.spark.deploy.ExternalShuffleService
+import org.apache.spark.deploy.mesos.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.network.client.{RpcResponseCallback, TransportClient}
+import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler
+import org.apache.spark.network.shuffle.protocol.BlockTransferMessage
+import org.apache.spark.network.shuffle.protocol.mesos.{RegisterDriver, ShuffleServiceHeartbeat}
+import org.apache.spark.network.util.TransportConf
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * An RPC endpoint that receives registration requests from Spark drivers running on Mesos.
+ * It detects driver termination and calls the cleanup callback to [[ExternalShuffleService]].
+ */
+private[mesos] class MesosExternalShuffleBlockHandler(
+    transportConf: TransportConf,
+    cleanerIntervalS: Long)
+  extends ExternalShuffleBlockHandler(transportConf, null) with Logging {
+
+  ThreadUtils.newDaemonSingleThreadScheduledExecutor("shuffle-cleaner-watcher")
+    .scheduleAtFixedRate(new CleanerThread(), 0, cleanerIntervalS, TimeUnit.SECONDS)
+
+  // Stores a map of app id to app state (timeout value and last heartbeat)
+  private val connectedApps = new ConcurrentHashMap[String, AppState]()
+
+  protected override def handleMessage(
+      message: BlockTransferMessage,
+      client: TransportClient,
+      callback: RpcResponseCallback): Unit = {
+    message match {
+      case RegisterDriverParam(appId, appState) =>
+        val address = client.getSocketAddress
+        val timeout = appState.heartbeatTimeout
+        logInfo(s"Received registration request from app $appId (remote address $address, " +
+          s"heartbeat timeout $timeout ms).")
+        if (connectedApps.containsKey(appId)) {
+          logWarning(s"Received a registration request from app $appId, but it was already " +
+            s"registered")
+        }
+        connectedApps.put(appId, appState)
+        callback.onSuccess(ByteBuffer.allocate(0))
+      case Heartbeat(appId) =>
+        val address = client.getSocketAddress
+        Option(connectedApps.get(appId)) match {
+          case Some(existingAppState) =>
+            logTrace(s"Received ShuffleServiceHeartbeat from app '$appId' (remote " +
+              s"address $address).")
+            existingAppState.lastHeartbeat = System.nanoTime()
+          case None =>
+            logWarning(s"Received ShuffleServiceHeartbeat from an unknown app (remote " +
+              s"address $address, appId '$appId').")
+        }
+      case _ => super.handleMessage(message, client, callback)
+    }
+  }
+
+  /** An extractor object for matching [[RegisterDriver]] message. */
+  private object RegisterDriverParam {
+    def unapply(r: RegisterDriver): Option[(String, AppState)] =
+      Some((r.getAppId, new AppState(r.getHeartbeatTimeoutMs, System.nanoTime())))
+  }
+
+  private object Heartbeat {
+    def unapply(h: ShuffleServiceHeartbeat): Option[String] = Some(h.getAppId)
+  }
+
+  private class AppState(val heartbeatTimeout: Long, @volatile var lastHeartbeat: Long)
+
+  private class CleanerThread extends Runnable {
+    override def run(): Unit = {
+      val now = System.nanoTime()
+      connectedApps.asScala.foreach { case (appId, appState) =>
+        if (now - appState.lastHeartbeat > appState.heartbeatTimeout * 1000 * 1000) {
+          logInfo(s"Application $appId timed out. Removing shuffle files.")
+          connectedApps.remove(appId)
+          applicationRemoved(appId, true)
+        }
+      }
+    }
+  }
+}
+
+/**
+ * A wrapper of [[ExternalShuffleService]] that provides an additional endpoint for drivers
+ * to associate with. This allows the shuffle service to detect when a driver is terminated
+ * and can clean up the associated shuffle files.
+ */
+private[mesos] class MesosExternalShuffleService(conf: SparkConf, securityManager: SecurityManager)
+  extends ExternalShuffleService(conf, securityManager) {
+
+  protected override def newShuffleBlockHandler(
+      conf: TransportConf): ExternalShuffleBlockHandler = {
+    val cleanerIntervalS = this.conf.get(SHUFFLE_CLEANER_INTERVAL_S)
+    new MesosExternalShuffleBlockHandler(conf, cleanerIntervalS)
+  }
+}
+
+private[spark] object MesosExternalShuffleService extends Logging {
+
+  def main(args: Array[String]): Unit = {
+    ExternalShuffleService.main(args,
+      (conf: SparkConf, sm: SecurityManager) => new MesosExternalShuffleService(conf, sm))
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala
new file mode 100644
index 0000000..19e2533
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala
@@ -0,0 +1,59 @@
+/*
+ * 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.mesos
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.spark.internal.config.ConfigBuilder
+
+package object config {
+
+  /* Common app configuration. */
+
+  private[spark] val SHUFFLE_CLEANER_INTERVAL_S =
+    ConfigBuilder("spark.shuffle.cleaner.interval")
+      .timeConf(TimeUnit.SECONDS)
+      .createWithDefaultString("30s")
+
+  private[spark] val RECOVERY_MODE =
+    ConfigBuilder("spark.deploy.recoveryMode")
+      .stringConf
+      .createWithDefault("NONE")
+
+  private[spark] val DISPATCHER_WEBUI_URL =
+    ConfigBuilder("spark.mesos.dispatcher.webui.url")
+      .doc("Set the Spark Mesos dispatcher webui_url for interacting with the " +
+        "framework. If unset it will point to Spark's internal web UI.")
+      .stringConf
+      .createOptional
+
+  private[spark] val ZOOKEEPER_URL =
+    ConfigBuilder("spark.deploy.zookeeper.url")
+      .doc("When `spark.deploy.recoveryMode` is set to ZOOKEEPER, this " +
+        "configuration is used to set the zookeeper URL to connect to.")
+      .stringConf
+      .createOptional
+
+  private[spark] val HISTORY_SERVER_URL =
+    ConfigBuilder("spark.mesos.dispatcher.historyServer.url")
+      .doc("Set the URL of the history server. The dispatcher will then " +
+        "link each driver to its entry in the history server.")
+      .stringConf
+      .createOptional
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala
new file mode 100644
index 0000000..cd98110
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.mesos.ui
+
+import javax.servlet.http.HttpServletRequest
+
+import scala.xml.Node
+
+import org.apache.spark.deploy.Command
+import org.apache.spark.deploy.mesos.MesosDriverDescription
+import org.apache.spark.scheduler.cluster.mesos.{MesosClusterRetryState, MesosClusterSubmissionState}
+import org.apache.spark.ui.{UIUtils, WebUIPage}
+
+private[ui] class DriverPage(parent: MesosClusterUI) extends WebUIPage("driver") {
+
+  override def render(request: HttpServletRequest): Seq[Node] = {
+    val driverId = request.getParameter("id")
+    require(driverId != null && driverId.nonEmpty, "Missing id parameter")
+
+    val state = parent.scheduler.getDriverState(driverId)
+    if (state.isEmpty) {
+      val content =
+        <div>
+          <p>Cannot find driver {driverId}</p>
+        </div>
+      return UIUtils.basicSparkPage(content, s"Details for Job $driverId")
+    }
+    val driverState = state.get
+    val driverHeaders = Seq("Driver property", "Value")
+    val schedulerHeaders = Seq("Scheduler property", "Value")
+    val commandEnvHeaders = Seq("Command environment variable", "Value")
+    val launchedHeaders = Seq("Launched property", "Value")
+    val commandHeaders = Seq("Command property", "Value")
+    val retryHeaders = Seq("Last failed status", "Next retry time", "Retry count")
+    val driverDescription = Iterable.apply(driverState.description)
+    val submissionState = Iterable.apply(driverState.submissionState)
+    val command = Iterable.apply(driverState.description.command)
+    val schedulerProperties = Iterable.apply(driverState.description.conf.getAll.toMap)
+    val commandEnv = Iterable.apply(driverState.description.command.environment)
+    val driverTable =
+      UIUtils.listingTable(driverHeaders, driverRow, driverDescription)
+    val commandTable =
+      UIUtils.listingTable(commandHeaders, commandRow, command)
+    val commandEnvTable =
+      UIUtils.listingTable(commandEnvHeaders, propertiesRow, commandEnv)
+    val schedulerTable =
+      UIUtils.listingTable(schedulerHeaders, propertiesRow, schedulerProperties)
+    val launchedTable =
+      UIUtils.listingTable(launchedHeaders, launchedRow, submissionState)
+    val retryTable =
+      UIUtils.listingTable(
+        retryHeaders, retryRow, Iterable.apply(driverState.description.retryState))
+    val content =
+      <p>Driver state information for driver id {driverId}</p>
+        <a href={UIUtils.prependBaseUri("/")}>Back to Drivers</a>
+        <div class="row-fluid">
+          <div class="span12">
+            <h4>Driver state: {driverState.state}</h4>
+            <h4>Driver properties</h4>
+            {driverTable}
+            <h4>Driver command</h4>
+            {commandTable}
+            <h4>Driver command environment</h4>
+            {commandEnvTable}
+            <h4>Scheduler properties</h4>
+            {schedulerTable}
+            <h4>Launched state</h4>
+            {launchedTable}
+            <h4>Retry state</h4>
+            {retryTable}
+          </div>
+        </div>;
+
+    UIUtils.basicSparkPage(content, s"Details for Job $driverId")
+  }
+
+  private def launchedRow(submissionState: Option[MesosClusterSubmissionState]): Seq[Node] = {
+    submissionState.map { state =>
+      <tr>
+        <td>Mesos Slave ID</td>
+        <td>{state.slaveId.getValue}</td>
+      </tr>
+      <tr>
+        <td>Mesos Task ID</td>
+        <td>{state.taskId.getValue}</td>
+      </tr>
+      <tr>
+        <td>Launch Time</td>
+        <td>{state.startDate}</td>
+      </tr>
+      <tr>
+        <td>Finish Time</td>
+        <td>{state.finishDate.map(_.toString).getOrElse("")}</td>
+      </tr>
+      <tr>
+        <td>Last Task Status</td>
+        <td>{state.mesosTaskStatus.map(_.toString).getOrElse("")}</td>
+      </tr>
+    }.getOrElse(Seq[Node]())
+  }
+
+  private def propertiesRow(properties: collection.Map[String, String]): Seq[Node] = {
+    properties.map { case (k, v) =>
+      <tr>
+        <td>{k}</td><td>{v}</td>
+      </tr>
+    }.toSeq
+  }
+
+  private def commandRow(command: Command): Seq[Node] = {
+    <tr>
+      <td>Main class</td><td>{command.mainClass}</td>
+    </tr>
+    <tr>
+      <td>Arguments</td><td>{command.arguments.mkString(" ")}</td>
+    </tr>
+    <tr>
+      <td>Class path entries</td><td>{command.classPathEntries.mkString(" ")}</td>
+    </tr>
+    <tr>
+      <td>Java options</td><td>{command.javaOpts.mkString((" "))}</td>
+    </tr>
+    <tr>
+      <td>Library path entries</td><td>{command.libraryPathEntries.mkString((" "))}</td>
+    </tr>
+  }
+
+  private def driverRow(driver: MesosDriverDescription): Seq[Node] = {
+    <tr>
+      <td>Name</td><td>{driver.name}</td>
+    </tr>
+    <tr>
+      <td>Id</td><td>{driver.submissionId}</td>
+    </tr>
+    <tr>
+      <td>Cores</td><td>{driver.cores}</td>
+    </tr>
+    <tr>
+      <td>Memory</td><td>{driver.mem}</td>
+    </tr>
+    <tr>
+      <td>Submitted</td><td>{driver.submissionDate}</td>
+    </tr>
+    <tr>
+      <td>Supervise</td><td>{driver.supervise}</td>
+    </tr>
+  }
+
+  private def retryRow(retryState: Option[MesosClusterRetryState]): Seq[Node] = {
+    retryState.map { state =>
+      <tr>
+        <td>
+          {state.lastFailureStatus}
+        </td>
+        <td>
+          {state.nextRetry}
+        </td>
+        <td>
+          {state.retries}
+        </td>
+      </tr>
+    }.getOrElse(Seq[Node]())
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala
new file mode 100644
index 0000000..13ba7d3
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala
@@ -0,0 +1,136 @@
+/*
+ * 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.mesos.ui
+
+import javax.servlet.http.HttpServletRequest
+
+import scala.xml.Node
+
+import org.apache.mesos.Protos.TaskStatus
+
+import org.apache.spark.deploy.mesos.config._
+import org.apache.spark.deploy.mesos.MesosDriverDescription
+import org.apache.spark.scheduler.cluster.mesos.MesosClusterSubmissionState
+import org.apache.spark.ui.{UIUtils, WebUIPage}
+
+private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage("") {
+  private val historyServerURL = parent.conf.get(HISTORY_SERVER_URL)
+
+  def render(request: HttpServletRequest): Seq[Node] = {
+    val state = parent.scheduler.getSchedulerState()
+
+    val driverHeader = Seq("Driver ID")
+    val historyHeader = historyServerURL.map(url => Seq("History")).getOrElse(Nil)
+    val submissionHeader = Seq("Submit Date", "Main Class", "Driver Resources")
+
+    val queuedHeaders = driverHeader ++ submissionHeader
+    val driverHeaders = driverHeader ++ historyHeader ++ submissionHeader ++
+      Seq("Start Date", "Mesos Slave ID", "State")
+    val retryHeaders = Seq("Driver ID", "Submit Date", "Description") ++
+      Seq("Last Failed Status", "Next Retry Time", "Attempt Count")
+    val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers)
+    val launchedTable = UIUtils.listingTable(driverHeaders, driverRow, state.launchedDrivers)
+    val finishedTable = UIUtils.listingTable(driverHeaders, driverRow, state.finishedDrivers)
+    val retryTable = UIUtils.listingTable(retryHeaders, retryRow, state.pendingRetryDrivers)
+    val content =
+      <p>Mesos Framework ID: {state.frameworkId}</p>
+      <div class="row-fluid">
+        <div class="span12">
+          <h4>Queued Drivers:</h4>
+          {queuedTable}
+          <h4>Launched Drivers:</h4>
+          {launchedTable}
+          <h4>Finished Drivers:</h4>
+          {finishedTable}
+          <h4>Supervise drivers waiting for retry:</h4>
+          {retryTable}
+        </div>
+      </div>;
+    UIUtils.basicSparkPage(content, "Spark Drivers for Mesos cluster")
+  }
+
+  private def queuedRow(submission: MesosDriverDescription): Seq[Node] = {
+    val id = submission.submissionId
+    <tr>
+      <td><a href={s"driver?id=$id"}>{id}</a></td>
+      <td>{submission.submissionDate}</td>
+      <td>{submission.command.mainClass}</td>
+      <td>cpus: {submission.cores}, mem: {submission.mem}</td>
+    </tr>
+  }
+
+  private def driverRow(state: MesosClusterSubmissionState): Seq[Node] = {
+    val id = state.driverDescription.submissionId
+
+    val historyCol = if (historyServerURL.isDefined) {
+      <td>
+        <a href={s"${historyServerURL.get}/history/${state.frameworkId}"}>
+          {state.frameworkId}
+        </a>
+      </td>
+    } else Nil
+
+    <tr>
+      <td><a href={s"driver?id=$id"}>{id}</a></td>
+      {historyCol}
+      <td>{state.driverDescription.submissionDate}</td>
+      <td>{state.driverDescription.command.mainClass}</td>
+      <td>cpus: {state.driverDescription.cores}, mem: {state.driverDescription.mem}</td>
+      <td>{state.startDate}</td>
+      <td>{state.slaveId.getValue}</td>
+      <td>{stateString(state.mesosTaskStatus)}</td>
+    </tr>
+  }
+
+  private def retryRow(submission: MesosDriverDescription): Seq[Node] = {
+    val id = submission.submissionId
+    <tr>
+      <td><a href={s"driver?id=$id"}>{id}</a></td>
+      <td>{submission.submissionDate}</td>
+      <td>{submission.command.mainClass}</td>
+      <td>{submission.retryState.get.lastFailureStatus}</td>
+      <td>{submission.retryState.get.nextRetry}</td>
+      <td>{submission.retryState.get.retries}</td>
+    </tr>
+  }
+
+  private def stateString(status: Option[TaskStatus]): String = {
+    if (status.isEmpty) {
+      return ""
+    }
+    val sb = new StringBuilder
+    val s = status.get
+    sb.append(s"State: ${s.getState}")
+    if (status.get.hasMessage) {
+      sb.append(s", Message: ${s.getMessage}")
+    }
+    if (status.get.hasHealthy) {
+      sb.append(s", Healthy: ${s.getHealthy}")
+    }
+    if (status.get.hasSource) {
+      sb.append(s", Source: ${s.getSource}")
+    }
+    if (status.get.hasReason) {
+      sb.append(s", Reason: ${s.getReason}")
+    }
+    if (status.get.hasTimestamp) {
+      sb.append(s", Time: ${s.getTimestamp}")
+    }
+    sb.toString()
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala
new file mode 100644
index 0000000..6049789
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.mesos.ui
+
+import org.apache.spark.{SecurityManager, SparkConf}
+import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler
+import org.apache.spark.ui.{SparkUI, WebUI}
+import org.apache.spark.ui.JettyUtils._
+
+/**
+ * UI that displays driver results from the [[org.apache.spark.deploy.mesos.MesosClusterDispatcher]]
+ */
+private[spark] class MesosClusterUI(
+    securityManager: SecurityManager,
+    port: Int,
+    val conf: SparkConf,
+    dispatcherPublicAddress: String,
+    val scheduler: MesosClusterScheduler)
+  extends WebUI(securityManager, securityManager.getSSLOptions("mesos"), port, conf) {
+
+  initialize()
+
+  def activeWebUiUrl: String = "http://" + dispatcherPublicAddress + ":" + boundPort
+
+  override def initialize() {
+    attachPage(new MesosClusterPage(this))
+    attachPage(new DriverPage(this))
+    attachHandler(createStaticHandler(MesosClusterUI.STATIC_RESOURCE_DIR, "/static"))
+  }
+}
+
+private object MesosClusterUI {
+  val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
new file mode 100644
index 0000000..ff60b88
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
@@ -0,0 +1,156 @@
+/*
+ * 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.rest.mesos
+
+import java.io.File
+import java.text.SimpleDateFormat
+import java.util.{Date, Locale}
+import java.util.concurrent.atomic.AtomicLong
+import javax.servlet.http.HttpServletResponse
+
+import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf}
+import org.apache.spark.deploy.Command
+import org.apache.spark.deploy.mesos.MesosDriverDescription
+import org.apache.spark.deploy.rest._
+import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler
+import org.apache.spark.util.Utils
+
+/**
+ * A server that responds to requests submitted by the [[RestSubmissionClient]].
+ * All requests are forwarded to
+ * [[org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler]].
+ * This is intended to be used in Mesos cluster mode only.
+ * For more details about the REST submission please refer to [[RestSubmissionServer]] javadocs.
+ */
+private[spark] class MesosRestServer(
+    host: String,
+    requestedPort: Int,
+    masterConf: SparkConf,
+    scheduler: MesosClusterScheduler)
+  extends RestSubmissionServer(host, requestedPort, masterConf) {
+
+  protected override val submitRequestServlet =
+    new MesosSubmitRequestServlet(scheduler, masterConf)
+  protected override val killRequestServlet =
+    new MesosKillRequestServlet(scheduler, masterConf)
+  protected override val statusRequestServlet =
+    new MesosStatusRequestServlet(scheduler, masterConf)
+}
+
+private[mesos] class MesosSubmitRequestServlet(
+    scheduler: MesosClusterScheduler,
+    conf: SparkConf)
+  extends SubmitRequestServlet {
+
+  private val DEFAULT_SUPERVISE = false
+  private val DEFAULT_MEMORY = Utils.DEFAULT_DRIVER_MEM_MB // mb
+  private val DEFAULT_CORES = 1.0
+
+  private val nextDriverNumber = new AtomicLong(0)
+  // For application IDs
+  private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US)
+  private def newDriverId(submitDate: Date): String =
+    f"driver-${createDateFormat.format(submitDate)}-${nextDriverNumber.incrementAndGet()}%04d"
+
+  /**
+   * Build a driver description from the fields specified in the submit request.
+   *
+   * This involves constructing a command that launches a mesos framework for the job.
+   * This does not currently consider fields used by python applications since python
+   * is not supported in mesos cluster mode yet.
+   */
+  private def buildDriverDescription(request: CreateSubmissionRequest): MesosDriverDescription = {
+    // Required fields, including the main class because python is not yet supported
+    val appResource = Option(request.appResource).getOrElse {
+      throw new SubmitRestMissingFieldException("Application jar is missing.")
+    }
+    val mainClass = Option(request.mainClass).getOrElse {
+      throw new SubmitRestMissingFieldException("Main class is missing.")
+    }
+
+    // Optional fields
+    val sparkProperties = request.sparkProperties
+    val driverExtraJavaOptions = sparkProperties.get("spark.driver.extraJavaOptions")
+    val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath")
+    val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath")
+    val superviseDriver = sparkProperties.get("spark.driver.supervise")
+    val driverMemory = sparkProperties.get("spark.driver.memory")
+    val driverCores = sparkProperties.get("spark.driver.cores")
+    val appArgs = request.appArgs
+    val environmentVariables = request.environmentVariables
+    val name = request.sparkProperties.getOrElse("spark.app.name", mainClass)
+
+    // Construct driver description
+    val conf = new SparkConf(false).setAll(sparkProperties)
+    val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator))
+    val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator))
+    val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty)
+    val sparkJavaOpts = Utils.sparkJavaOpts(conf)
+    val javaOpts = sparkJavaOpts ++ extraJavaOpts
+    val command = new Command(
+      mainClass, appArgs, environmentVariables, extraClassPath, extraLibraryPath, javaOpts)
+    val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE)
+    val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY)
+    val actualDriverCores = driverCores.map(_.toDouble).getOrElse(DEFAULT_CORES)
+    val submitDate = new Date()
+    val submissionId = newDriverId(submitDate)
+
+    new MesosDriverDescription(
+      name, appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver,
+      command, request.sparkProperties, submissionId, submitDate)
+  }
+
+  protected override def handleSubmit(
+      requestMessageJson: String,
+      requestMessage: SubmitRestProtocolMessage,
+      responseServlet: HttpServletResponse): SubmitRestProtocolResponse = {
+    requestMessage match {
+      case submitRequest: CreateSubmissionRequest =>
+        val driverDescription = buildDriverDescription(submitRequest)
+        val s = scheduler.submitDriver(driverDescription)
+        s.serverSparkVersion = sparkVersion
+        val unknownFields = findUnknownFields(requestMessageJson, requestMessage)
+        if (unknownFields.nonEmpty) {
+          // If there are fields that the server does not know about, warn the client
+          s.unknownFields = unknownFields
+        }
+        s
+      case unexpected =>
+        responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST)
+        handleError(s"Received message of unexpected type ${unexpected.messageType}.")
+    }
+  }
+}
+
+private[mesos] class MesosKillRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf)
+  extends KillRequestServlet {
+  protected override def handleKill(submissionId: String): KillSubmissionResponse = {
+    val k = scheduler.killDriver(submissionId)
+    k.serverSparkVersion = sparkVersion
+    k
+  }
+}
+
+private[mesos] class MesosStatusRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf)
+  extends StatusRequestServlet {
+  protected override def handleStatus(submissionId: String): SubmissionStatusResponse = {
+    val d = scheduler.getDriverStatus(submissionId)
+    d.serverSparkVersion = sparkVersion
+    d
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
new file mode 100644
index 0000000..ee9149c
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
@@ -0,0 +1,131 @@
+/*
+ * 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.executor
+
+import java.nio.ByteBuffer
+
+import scala.collection.JavaConverters._
+
+import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver}
+import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _}
+import org.apache.mesos.protobuf.ByteString
+
+import org.apache.spark.{SparkConf, SparkEnv, TaskState}
+import org.apache.spark.TaskState
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.scheduler.cluster.mesos.{MesosSchedulerUtils, MesosTaskLaunchData}
+import org.apache.spark.util.Utils
+
+private[spark] class MesosExecutorBackend
+  extends MesosExecutor
+  with MesosSchedulerUtils // TODO: fix
+  with ExecutorBackend
+  with Logging {
+
+  var executor: Executor = null
+  var driver: ExecutorDriver = null
+
+  override def statusUpdate(taskId: Long, state: TaskState.TaskState, data: ByteBuffer) {
+    val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build()
+    driver.sendStatusUpdate(MesosTaskStatus.newBuilder()
+      .setTaskId(mesosTaskId)
+      .setState(taskStateToMesos(state))
+      .setData(ByteString.copyFrom(data))
+      .build())
+  }
+
+  override def registered(
+      driver: ExecutorDriver,
+      executorInfo: ExecutorInfo,
+      frameworkInfo: FrameworkInfo,
+      slaveInfo: SlaveInfo) {
+
+    // Get num cores for this task from ExecutorInfo, created in MesosSchedulerBackend.
+    val cpusPerTask = executorInfo.getResourcesList.asScala
+      .find(_.getName == "cpus")
+      .map(_.getScalar.getValue.toInt)
+      .getOrElse(0)
+    val executorId = executorInfo.getExecutorId.getValue
+
+    logInfo(s"Registered with Mesos as executor ID $executorId with $cpusPerTask cpus")
+    this.driver = driver
+    // Set a context class loader to be picked up by the serializer. Without this call
+    // the serializer would default to the null class loader, and fail to find Spark classes
+    // See SPARK-10986.
+    Thread.currentThread().setContextClassLoader(this.getClass.getClassLoader)
+
+    val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) ++
+      Seq[(String, String)](("spark.app.id", frameworkInfo.getId.getValue))
+    val conf = new SparkConf(loadDefaults = true).setAll(properties)
+    val port = conf.getInt("spark.executor.port", 0)
+    val env = SparkEnv.createExecutorEnv(
+      conf, executorId, slaveInfo.getHostname, port, cpusPerTask, None, isLocal = false)
+
+    executor = new Executor(
+      executorId,
+      slaveInfo.getHostname,
+      env)
+  }
+
+  override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) {
+    val taskId = taskInfo.getTaskId.getValue.toLong
+    val taskData = MesosTaskLaunchData.fromByteString(taskInfo.getData)
+    if (executor == null) {
+      logError("Received launchTask but executor was null")
+    } else {
+      SparkHadoopUtil.get.runAsSparkUser { () =>
+        executor.launchTask(this, taskId = taskId, attemptNumber = taskData.attemptNumber,
+          taskInfo.getName, taskData.serializedTask)
+      }
+    }
+  }
+
+  override def error(d: ExecutorDriver, message: String) {
+    logError("Error from Mesos: " + message)
+  }
+
+  override def killTask(d: ExecutorDriver, t: TaskID) {
+    if (executor == null) {
+      logError("Received KillTask but executor was null")
+    } else {
+      // TODO: Determine the 'interruptOnCancel' property set for the given job.
+      executor.killTask(t.getValue.toLong, interruptThread = false)
+    }
+  }
+
+  override def reregistered(d: ExecutorDriver, p2: SlaveInfo) {}
+
+  override def disconnected(d: ExecutorDriver) {}
+
+  override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]) {}
+
+  override def shutdown(d: ExecutorDriver) {}
+}
+
+/**
+ * Entry point for Mesos executor.
+ */
+private[spark] object MesosExecutorBackend extends Logging {
+  def main(args: Array[String]) {
+    Utils.initDaemon(log)
+    // Create a new Executor and start it running
+    val runner = new MesosExecutorBackend()
+    new MesosExecutorDriver(runner).run()
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala
new file mode 100644
index 0000000..ed29b34
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala
@@ -0,0 +1,64 @@
+/*
+ * 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.mesos
+
+import org.apache.spark.{SparkContext, SparkException}
+import org.apache.spark.internal.config._
+import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
+
+/**
+ * Cluster Manager for creation of Yarn scheduler and backend
+ */
+private[spark] class MesosClusterManager extends ExternalClusterManager {
+  private val MESOS_REGEX = """mesos://(.*)""".r
+
+  override def canCreate(masterURL: String): Boolean = {
+    masterURL.startsWith("mesos")
+  }
+
+  override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = {
+    new TaskSchedulerImpl(sc)
+  }
+
+  override def createSchedulerBackend(sc: SparkContext,
+      masterURL: String,
+      scheduler: TaskScheduler): SchedulerBackend = {
+    require(!sc.conf.get(IO_ENCRYPTION_ENABLED),
+      "I/O encryption is currently not supported in Mesos.")
+
+    val mesosUrl = MESOS_REGEX.findFirstMatchIn(masterURL).get.group(1)
+    val coarse = sc.conf.getBoolean("spark.mesos.coarse", defaultValue = true)
+    if (coarse) {
+      new MesosCoarseGrainedSchedulerBackend(
+        scheduler.asInstanceOf[TaskSchedulerImpl],
+        sc,
+        mesosUrl,
+        sc.env.securityManager)
+    } else {
+      new MesosFineGrainedSchedulerBackend(
+        scheduler.asInstanceOf[TaskSchedulerImpl],
+        sc,
+        mesosUrl)
+    }
+  }
+
+  override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = {
+    scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend)
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala
new file mode 100644
index 0000000..61ab3e8
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala
@@ -0,0 +1,134 @@
+/*
+ * 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.mesos
+
+import scala.collection.JavaConverters._
+
+import org.apache.curator.framework.CuratorFramework
+import org.apache.zookeeper.CreateMode
+import org.apache.zookeeper.KeeperException.NoNodeException
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkCuratorUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+/**
+ * Persistence engine factory that is responsible for creating new persistence engines
+ * to store Mesos cluster mode state.
+ */
+private[spark] abstract class MesosClusterPersistenceEngineFactory(conf: SparkConf) {
+  def createEngine(path: String): MesosClusterPersistenceEngine
+}
+
+/**
+ * Mesos cluster persistence engine is responsible for persisting Mesos cluster mode
+ * specific state, so that on failover all the state can be recovered and the scheduler
+ * can resume managing the drivers.
+ */
+private[spark] trait MesosClusterPersistenceEngine {
+  def persist(name: String, obj: Object): Unit
+  def expunge(name: String): Unit
+  def fetch[T](name: String): Option[T]
+  def fetchAll[T](): Iterable[T]
+}
+
+/**
+ * Zookeeper backed persistence engine factory.
+ * All Zk engines created from this factory shares the same Zookeeper client, so
+ * all of them reuses the same connection pool.
+ */
+private[spark] class ZookeeperMesosClusterPersistenceEngineFactory(conf: SparkConf)
+  extends MesosClusterPersistenceEngineFactory(conf) with Logging {
+
+  lazy val zk = SparkCuratorUtil.newClient(conf)
+
+  def createEngine(path: String): MesosClusterPersistenceEngine = {
+    new ZookeeperMesosClusterPersistenceEngine(path, zk, conf)
+  }
+}
+
+/**
+ * Black hole persistence engine factory that creates black hole
+ * persistence engines, which stores nothing.
+ */
+private[spark] class BlackHoleMesosClusterPersistenceEngineFactory
+  extends MesosClusterPersistenceEngineFactory(null) {
+  def createEngine(path: String): MesosClusterPersistenceEngine = {
+    new BlackHoleMesosClusterPersistenceEngine
+  }
+}
+
+/**
+ * Black hole persistence engine that stores nothing.
+ */
+private[spark] class BlackHoleMesosClusterPersistenceEngine extends MesosClusterPersistenceEngine {
+  override def persist(name: String, obj: Object): Unit = {}
+  override def fetch[T](name: String): Option[T] = None
+  override def expunge(name: String): Unit = {}
+  override def fetchAll[T](): Iterable[T] = Iterable.empty[T]
+}
+
+/**
+ * Zookeeper based Mesos cluster persistence engine, that stores cluster mode state
+ * into Zookeeper. Each engine object is operating under one folder in Zookeeper, but
+ * reuses a shared Zookeeper client.
+ */
+private[spark] class ZookeeperMesosClusterPersistenceEngine(
+    baseDir: String,
+    zk: CuratorFramework,
+    conf: SparkConf)
+  extends MesosClusterPersistenceEngine with Logging {
+  private val WORKING_DIR =
+    conf.get("spark.deploy.zookeeper.dir", "/spark_mesos_dispatcher") + "/" + baseDir
+
+  SparkCuratorUtil.mkdir(zk, WORKING_DIR)
+
+  def path(name: String): String = {
+    WORKING_DIR + "/" + name
+  }
+
+  override def expunge(name: String): Unit = {
+    zk.delete().forPath(path(name))
+  }
+
+  override def persist(name: String, obj: Object): Unit = {
+    val serialized = Utils.serialize(obj)
+    val zkPath = path(name)
+    zk.create().withMode(CreateMode.PERSISTENT).forPath(zkPath, serialized)
+  }
+
+  override def fetch[T](name: String): Option[T] = {
+    val zkPath = path(name)
+
+    try {
+      val fileData = zk.getData().forPath(zkPath)
+      Some(Utils.deserialize[T](fileData))
+    } catch {
+      case e: NoNodeException => None
+      case e: Exception =>
+        logWarning("Exception while reading persisted file, deleting", e)
+        zk.delete().forPath(zkPath)
+        None
+    }
+  }
+
+  override def fetchAll[T](): Iterable[T] = {
+    zk.getChildren.forPath(WORKING_DIR).asScala.flatMap(fetch[T])
+  }
+}


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


[14/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala
new file mode 100644
index 0000000..f384290
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala
@@ -0,0 +1,740 @@
+/*
+ * 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.mesos
+
+import java.io.File
+import java.util.{Collections, Date, List => JList}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.mesos.{Scheduler, SchedulerDriver}
+import org.apache.mesos.Protos.{TaskState => MesosTaskState, _}
+import org.apache.mesos.Protos.Environment.Variable
+import org.apache.mesos.Protos.TaskStatus.Reason
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkException, TaskState}
+import org.apache.spark.deploy.mesos.MesosDriverDescription
+import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse}
+import org.apache.spark.metrics.MetricsSystem
+import org.apache.spark.util.Utils
+
+/**
+ * Tracks the current state of a Mesos Task that runs a Spark driver.
+ * @param driverDescription Submitted driver description from
+ * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]]
+ * @param taskId Mesos TaskID generated for the task
+ * @param slaveId Slave ID that the task is assigned to
+ * @param mesosTaskStatus The last known task status update.
+ * @param startDate The date the task was launched
+ * @param finishDate The date the task finished
+ * @param frameworkId Mesos framework ID the task registers with
+ */
+private[spark] class MesosClusterSubmissionState(
+    val driverDescription: MesosDriverDescription,
+    val taskId: TaskID,
+    val slaveId: SlaveID,
+    var mesosTaskStatus: Option[TaskStatus],
+    var startDate: Date,
+    var finishDate: Option[Date],
+    val frameworkId: String)
+  extends Serializable {
+
+  def copy(): MesosClusterSubmissionState = {
+    new MesosClusterSubmissionState(
+      driverDescription, taskId, slaveId, mesosTaskStatus, startDate, finishDate, frameworkId)
+  }
+}
+
+/**
+ * Tracks the retry state of a driver, which includes the next time it should be scheduled
+ * and necessary information to do exponential backoff.
+ * This class is not thread-safe, and we expect the caller to handle synchronizing state.
+ *
+ * @param lastFailureStatus Last Task status when it failed.
+ * @param retries Number of times it has been retried.
+ * @param nextRetry Time at which it should be retried next
+ * @param waitTime The amount of time driver is scheduled to wait until next retry.
+ */
+private[spark] class MesosClusterRetryState(
+    val lastFailureStatus: TaskStatus,
+    val retries: Int,
+    val nextRetry: Date,
+    val waitTime: Int) extends Serializable {
+  def copy(): MesosClusterRetryState =
+    new MesosClusterRetryState(lastFailureStatus, retries, nextRetry, waitTime)
+}
+
+/**
+ * The full state of the cluster scheduler, currently being used for displaying
+ * information on the UI.
+ *
+ * @param frameworkId Mesos Framework id for the cluster scheduler.
+ * @param masterUrl The Mesos master url
+ * @param queuedDrivers All drivers queued to be launched
+ * @param launchedDrivers All launched or running drivers
+ * @param finishedDrivers All terminated drivers
+ * @param pendingRetryDrivers All drivers pending to be retried
+ */
+private[spark] class MesosClusterSchedulerState(
+    val frameworkId: String,
+    val masterUrl: Option[String],
+    val queuedDrivers: Iterable[MesosDriverDescription],
+    val launchedDrivers: Iterable[MesosClusterSubmissionState],
+    val finishedDrivers: Iterable[MesosClusterSubmissionState],
+    val pendingRetryDrivers: Iterable[MesosDriverDescription])
+
+/**
+ * The full state of a Mesos driver, that is being used to display driver information on the UI.
+ */
+private[spark] class MesosDriverState(
+    val state: String,
+    val description: MesosDriverDescription,
+    val submissionState: Option[MesosClusterSubmissionState] = None)
+
+/**
+ * A Mesos scheduler that is responsible for launching submitted Spark drivers in cluster mode
+ * as Mesos tasks in a Mesos cluster.
+ * All drivers are launched asynchronously by the framework, which will eventually be launched
+ * by one of the slaves in the cluster. The results of the driver will be stored in slave's task
+ * sandbox which is accessible by visiting the Mesos UI.
+ * This scheduler supports recovery by persisting all its state and performs task reconciliation
+ * on recover, which gets all the latest state for all the drivers from Mesos master.
+ */
+private[spark] class MesosClusterScheduler(
+    engineFactory: MesosClusterPersistenceEngineFactory,
+    conf: SparkConf)
+  extends Scheduler with MesosSchedulerUtils {
+  var frameworkUrl: String = _
+  private val metricsSystem =
+    MetricsSystem.createMetricsSystem("mesos_cluster", conf, new SecurityManager(conf))
+  private val master = conf.get("spark.master")
+  private val appName = conf.get("spark.app.name")
+  private val queuedCapacity = conf.getInt("spark.mesos.maxDrivers", 200)
+  private val retainedDrivers = conf.getInt("spark.mesos.retainedDrivers", 200)
+  private val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute
+  private val useFetchCache = conf.getBoolean("spark.mesos.fetchCache.enable", false)
+  private val schedulerState = engineFactory.createEngine("scheduler")
+  private val stateLock = new Object()
+  private val finishedDrivers =
+    new mutable.ArrayBuffer[MesosClusterSubmissionState](retainedDrivers)
+  private var frameworkId: String = null
+  // Holds all the launched drivers and current launch state, keyed by driver id.
+  private val launchedDrivers = new mutable.HashMap[String, MesosClusterSubmissionState]()
+  // Holds a map of driver id to expected slave id that is passed to Mesos for reconciliation.
+  // All drivers that are loaded after failover are added here, as we need get the latest
+  // state of the tasks from Mesos.
+  private val pendingRecover = new mutable.HashMap[String, SlaveID]()
+  // Stores all the submitted drivers that hasn't been launched.
+  private val queuedDrivers = new ArrayBuffer[MesosDriverDescription]()
+  // All supervised drivers that are waiting to retry after termination.
+  private val pendingRetryDrivers = new ArrayBuffer[MesosDriverDescription]()
+  private val queuedDriversState = engineFactory.createEngine("driverQueue")
+  private val launchedDriversState = engineFactory.createEngine("launchedDrivers")
+  private val pendingRetryDriversState = engineFactory.createEngine("retryList")
+  // Flag to mark if the scheduler is ready to be called, which is until the scheduler
+  // is registered with Mesos master.
+  @volatile protected var ready = false
+  private var masterInfo: Option[MasterInfo] = None
+
+  def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse = {
+    val c = new CreateSubmissionResponse
+    if (!ready) {
+      c.success = false
+      c.message = "Scheduler is not ready to take requests"
+      return c
+    }
+
+    stateLock.synchronized {
+      if (isQueueFull()) {
+        c.success = false
+        c.message = "Already reached maximum submission size"
+        return c
+      }
+      c.submissionId = desc.submissionId
+      queuedDriversState.persist(desc.submissionId, desc)
+      queuedDrivers += desc
+      c.success = true
+    }
+    c
+  }
+
+  def killDriver(submissionId: String): KillSubmissionResponse = {
+    val k = new KillSubmissionResponse
+    if (!ready) {
+      k.success = false
+      k.message = "Scheduler is not ready to take requests"
+      return k
+    }
+    k.submissionId = submissionId
+    stateLock.synchronized {
+      // We look for the requested driver in the following places:
+      // 1. Check if submission is running or launched.
+      // 2. Check if it's still queued.
+      // 3. Check if it's in the retry list.
+      // 4. Check if it has already completed.
+      if (launchedDrivers.contains(submissionId)) {
+        val task = launchedDrivers(submissionId)
+        mesosDriver.killTask(task.taskId)
+        k.success = true
+        k.message = "Killing running driver"
+      } else if (removeFromQueuedDrivers(submissionId)) {
+        k.success = true
+        k.message = "Removed driver while it's still pending"
+      } else if (removeFromPendingRetryDrivers(submissionId)) {
+        k.success = true
+        k.message = "Removed driver while it's being retried"
+      } else if (finishedDrivers.exists(_.driverDescription.submissionId.equals(submissionId))) {
+        k.success = false
+        k.message = "Driver already terminated"
+      } else {
+        k.success = false
+        k.message = "Cannot find driver"
+      }
+    }
+    k
+  }
+
+  def getDriverStatus(submissionId: String): SubmissionStatusResponse = {
+    val s = new SubmissionStatusResponse
+    if (!ready) {
+      s.success = false
+      s.message = "Scheduler is not ready to take requests"
+      return s
+    }
+    s.submissionId = submissionId
+    stateLock.synchronized {
+      if (queuedDrivers.exists(_.submissionId.equals(submissionId))) {
+        s.success = true
+        s.driverState = "QUEUED"
+      } else if (launchedDrivers.contains(submissionId)) {
+        s.success = true
+        s.driverState = "RUNNING"
+        launchedDrivers(submissionId).mesosTaskStatus.foreach(state => s.message = state.toString)
+      } else if (finishedDrivers.exists(_.driverDescription.submissionId.equals(submissionId))) {
+        s.success = true
+        s.driverState = "FINISHED"
+        finishedDrivers
+          .find(d => d.driverDescription.submissionId.equals(submissionId)).get.mesosTaskStatus
+          .foreach(state => s.message = state.toString)
+      } else if (pendingRetryDrivers.exists(_.submissionId.equals(submissionId))) {
+        val status = pendingRetryDrivers.find(_.submissionId.equals(submissionId))
+          .get.retryState.get.lastFailureStatus
+        s.success = true
+        s.driverState = "RETRYING"
+        s.message = status.toString
+      } else {
+        s.success = false
+        s.driverState = "NOT_FOUND"
+      }
+    }
+    s
+  }
+
+  /**
+   * Gets the driver state to be displayed on the Web UI.
+   */
+  def getDriverState(submissionId: String): Option[MesosDriverState] = {
+    stateLock.synchronized {
+      queuedDrivers.find(_.submissionId.equals(submissionId))
+        .map(d => new MesosDriverState("QUEUED", d))
+        .orElse(launchedDrivers.get(submissionId)
+          .map(d => new MesosDriverState("RUNNING", d.driverDescription, Some(d))))
+        .orElse(finishedDrivers.find(_.driverDescription.submissionId.equals(submissionId))
+          .map(d => new MesosDriverState("FINISHED", d.driverDescription, Some(d))))
+        .orElse(pendingRetryDrivers.find(_.submissionId.equals(submissionId))
+          .map(d => new MesosDriverState("RETRYING", d)))
+    }
+  }
+
+  private def isQueueFull(): Boolean = launchedDrivers.size >= queuedCapacity
+
+  /**
+   * Recover scheduler state that is persisted.
+   * We still need to do task reconciliation to be up to date of the latest task states
+   * as it might have changed while the scheduler is failing over.
+   */
+  private def recoverState(): Unit = {
+    stateLock.synchronized {
+      launchedDriversState.fetchAll[MesosClusterSubmissionState]().foreach { state =>
+        launchedDrivers(state.taskId.getValue) = state
+        pendingRecover(state.taskId.getValue) = state.slaveId
+      }
+      queuedDriversState.fetchAll[MesosDriverDescription]().foreach(d => queuedDrivers += d)
+      // There is potential timing issue where a queued driver might have been launched
+      // but the scheduler shuts down before the queued driver was able to be removed
+      // from the queue. We try to mitigate this issue by walking through all queued drivers
+      // and remove if they're already launched.
+      queuedDrivers
+        .filter(d => launchedDrivers.contains(d.submissionId))
+        .foreach(d => removeFromQueuedDrivers(d.submissionId))
+      pendingRetryDriversState.fetchAll[MesosDriverDescription]()
+        .foreach(s => pendingRetryDrivers += s)
+      // TODO: Consider storing finished drivers so we can show them on the UI after
+      // failover. For now we clear the history on each recovery.
+      finishedDrivers.clear()
+    }
+  }
+
+  /**
+   * Starts the cluster scheduler and wait until the scheduler is registered.
+   * This also marks the scheduler to be ready for requests.
+   */
+  def start(): Unit = {
+    // TODO: Implement leader election to make sure only one framework running in the cluster.
+    val fwId = schedulerState.fetch[String]("frameworkId")
+    fwId.foreach { id =>
+      frameworkId = id
+    }
+    recoverState()
+    metricsSystem.registerSource(new MesosClusterSchedulerSource(this))
+    metricsSystem.start()
+    val driver = createSchedulerDriver(
+      master,
+      MesosClusterScheduler.this,
+      Utils.getCurrentUserName(),
+      appName,
+      conf,
+      Some(frameworkUrl),
+      Some(true),
+      Some(Integer.MAX_VALUE),
+      fwId)
+
+    startScheduler(driver)
+    ready = true
+  }
+
+  def stop(): Unit = {
+    ready = false
+    metricsSystem.report()
+    metricsSystem.stop()
+    mesosDriver.stop(true)
+  }
+
+  override def registered(
+      driver: SchedulerDriver,
+      newFrameworkId: FrameworkID,
+      masterInfo: MasterInfo): Unit = {
+    logInfo("Registered as framework ID " + newFrameworkId.getValue)
+    if (newFrameworkId.getValue != frameworkId) {
+      frameworkId = newFrameworkId.getValue
+      schedulerState.persist("frameworkId", frameworkId)
+    }
+    markRegistered()
+
+    stateLock.synchronized {
+      this.masterInfo = Some(masterInfo)
+      if (!pendingRecover.isEmpty) {
+        // Start task reconciliation if we need to recover.
+        val statuses = pendingRecover.collect {
+          case (taskId, slaveId) =>
+            val newStatus = TaskStatus.newBuilder()
+              .setTaskId(TaskID.newBuilder().setValue(taskId).build())
+              .setSlaveId(slaveId)
+              .setState(MesosTaskState.TASK_STAGING)
+              .build()
+            launchedDrivers.get(taskId).map(_.mesosTaskStatus.getOrElse(newStatus))
+              .getOrElse(newStatus)
+        }
+        // TODO: Page the status updates to avoid trying to reconcile
+        // a large amount of tasks at once.
+        driver.reconcileTasks(statuses.toSeq.asJava)
+      }
+    }
+  }
+
+  private def getDriverExecutorURI(desc: MesosDriverDescription): Option[String] = {
+    desc.conf.getOption("spark.executor.uri")
+      .orElse(desc.command.environment.get("SPARK_EXECUTOR_URI"))
+  }
+
+  private def getDriverFrameworkID(desc: MesosDriverDescription): String = {
+    s"${frameworkId}-${desc.submissionId}"
+  }
+
+  private def adjust[A, B](m: collection.Map[A, B], k: A, default: B)(f: B => B) = {
+    m.updated(k, f(m.getOrElse(k, default)))
+  }
+
+  private def getDriverEnvironment(desc: MesosDriverDescription): Environment = {
+    // TODO(mgummelt): Don't do this here.  This should be passed as a --conf
+    val commandEnv = adjust(desc.command.environment, "SPARK_SUBMIT_OPTS", "")(
+      v => s"$v -Dspark.mesos.driver.frameworkId=${getDriverFrameworkID(desc)}"
+    )
+
+    val env = desc.conf.getAllWithPrefix("spark.mesos.driverEnv.") ++ commandEnv
+
+    val envBuilder = Environment.newBuilder()
+    env.foreach { case (k, v) =>
+      envBuilder.addVariables(Variable.newBuilder().setName(k).setValue(v))
+    }
+    envBuilder.build()
+  }
+
+  private def getDriverUris(desc: MesosDriverDescription): List[CommandInfo.URI] = {
+    val confUris = List(conf.getOption("spark.mesos.uris"),
+      desc.conf.getOption("spark.mesos.uris"),
+      desc.conf.getOption("spark.submit.pyFiles")).flatMap(
+      _.map(_.split(",").map(_.trim))
+    ).flatten
+
+    val jarUrl = desc.jarUrl.stripPrefix("file:").stripPrefix("local:")
+
+    ((jarUrl :: confUris) ++ getDriverExecutorURI(desc).toList).map(uri =>
+      CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetchCache).build())
+  }
+
+  private def getDriverCommandValue(desc: MesosDriverDescription): String = {
+    val dockerDefined = desc.conf.contains("spark.mesos.executor.docker.image")
+    val executorUri = getDriverExecutorURI(desc)
+    // Gets the path to run spark-submit, and the path to the Mesos sandbox.
+    val (executable, sandboxPath) = if (dockerDefined) {
+      // Application jar is automatically downloaded in the mounted sandbox by Mesos,
+      // and the path to the mounted volume is stored in $MESOS_SANDBOX env variable.
+      ("./bin/spark-submit", "$MESOS_SANDBOX")
+    } else if (executorUri.isDefined) {
+      val folderBasename = executorUri.get.split('/').last.split('.').head
+
+      val entries = conf.getOption("spark.executor.extraLibraryPath")
+        .map(path => Seq(path) ++ desc.command.libraryPathEntries)
+        .getOrElse(desc.command.libraryPathEntries)
+
+      val prefixEnv = if (!entries.isEmpty) Utils.libraryPathEnvPrefix(entries) else ""
+
+      val cmdExecutable = s"cd $folderBasename*; $prefixEnv bin/spark-submit"
+      // Sandbox path points to the parent folder as we chdir into the folderBasename.
+      (cmdExecutable, "..")
+    } else {
+      val executorSparkHome = desc.conf.getOption("spark.mesos.executor.home")
+        .orElse(conf.getOption("spark.home"))
+        .orElse(Option(System.getenv("SPARK_HOME")))
+        .getOrElse {
+          throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!")
+        }
+      val cmdExecutable = new File(executorSparkHome, "./bin/spark-submit").getPath
+      // Sandbox points to the current directory by default with Mesos.
+      (cmdExecutable, ".")
+    }
+    val cmdOptions = generateCmdOption(desc, sandboxPath).mkString(" ")
+    val primaryResource = new File(sandboxPath, desc.jarUrl.split("/").last).toString()
+    val appArguments = desc.command.arguments.mkString(" ")
+
+    s"$executable $cmdOptions $primaryResource $appArguments"
+  }
+
+  private def buildDriverCommand(desc: MesosDriverDescription): CommandInfo = {
+    val builder = CommandInfo.newBuilder()
+    builder.setValue(getDriverCommandValue(desc))
+    builder.setEnvironment(getDriverEnvironment(desc))
+    builder.addAllUris(getDriverUris(desc).asJava)
+    builder.build()
+  }
+
+  private def generateCmdOption(desc: MesosDriverDescription, sandboxPath: String): Seq[String] = {
+    var options = Seq(
+      "--name", desc.conf.get("spark.app.name"),
+      "--master", s"mesos://${conf.get("spark.master")}",
+      "--driver-cores", desc.cores.toString,
+      "--driver-memory", s"${desc.mem}M")
+
+    // Assume empty main class means we're running python
+    if (!desc.command.mainClass.equals("")) {
+      options ++= Seq("--class", desc.command.mainClass)
+    }
+
+    desc.conf.getOption("spark.executor.memory").foreach { v =>
+      options ++= Seq("--executor-memory", v)
+    }
+    desc.conf.getOption("spark.cores.max").foreach { v =>
+      options ++= Seq("--total-executor-cores", v)
+    }
+    desc.conf.getOption("spark.submit.pyFiles").foreach { pyFiles =>
+      val formattedFiles = pyFiles.split(",")
+        .map { path => new File(sandboxPath, path.split("/").last).toString() }
+        .mkString(",")
+      options ++= Seq("--py-files", formattedFiles)
+    }
+
+    // --conf
+    val replicatedOptionsBlacklist = Set(
+      "spark.jars", // Avoids duplicate classes in classpath
+      "spark.submit.deployMode", // this would be set to `cluster`, but we need client
+      "spark.master" // this contains the address of the dispatcher, not master
+    )
+    val defaultConf = conf.getAllWithPrefix("spark.mesos.dispatcher.driverDefault.").toMap
+    val driverConf = desc.conf.getAll
+      .filter { case (key, _) => !replicatedOptionsBlacklist.contains(key) }
+      .toMap
+    (defaultConf ++ driverConf).foreach { case (key, value) =>
+      options ++= Seq("--conf", s""""$key=${shellEscape(value)}"""".stripMargin) }
+
+    options
+  }
+
+  /**
+   * Escape args for Unix-like shells, unless already quoted by the user.
+   * Based on: http://www.gnu.org/software/bash/manual/html_node/Double-Quotes.html
+   * and http://www.grymoire.com/Unix/Quote.html
+ *
+   * @param value argument
+   * @return escaped argument
+   */
+  private[scheduler] def shellEscape(value: String): String = {
+    val WrappedInQuotes = """^(".+"|'.+')$""".r
+    val ShellSpecialChars = (""".*([ '<>&|\?\*;!#\\(\)"$`]).*""").r
+    value match {
+      case WrappedInQuotes(c) => value // The user quoted his args, don't touch it!
+      case ShellSpecialChars(c) => "\"" + value.replaceAll("""(["`\$\\])""", """\\$1""") + "\""
+      case _: String => value // Don't touch harmless strings
+    }
+  }
+
+  private class ResourceOffer(
+      val offerId: OfferID,
+      val slaveId: SlaveID,
+      var resources: JList[Resource]) {
+    override def toString(): String = {
+      s"Offer id: ${offerId}, resources: ${resources}"
+    }
+  }
+
+  private def createTaskInfo(desc: MesosDriverDescription, offer: ResourceOffer): TaskInfo = {
+    val taskId = TaskID.newBuilder().setValue(desc.submissionId).build()
+
+    val (remainingResources, cpuResourcesToUse) =
+      partitionResources(offer.resources, "cpus", desc.cores)
+    val (finalResources, memResourcesToUse) =
+      partitionResources(remainingResources.asJava, "mem", desc.mem)
+    offer.resources = finalResources.asJava
+
+    val appName = desc.conf.get("spark.app.name")
+    val taskInfo = TaskInfo.newBuilder()
+      .setTaskId(taskId)
+      .setName(s"Driver for ${appName}")
+      .setSlaveId(offer.slaveId)
+      .setCommand(buildDriverCommand(desc))
+      .addAllResources(cpuResourcesToUse.asJava)
+      .addAllResources(memResourcesToUse.asJava)
+    taskInfo.setContainer(MesosSchedulerBackendUtil.containerInfo(desc.conf))
+    taskInfo.build
+  }
+
+  /**
+   * This method takes all the possible candidates and attempt to schedule them with Mesos offers.
+   * Every time a new task is scheduled, the afterLaunchCallback is called to perform post scheduled
+   * logic on each task.
+   */
+  private def scheduleTasks(
+      candidates: Seq[MesosDriverDescription],
+      afterLaunchCallback: (String) => Boolean,
+      currentOffers: List[ResourceOffer],
+      tasks: mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]): Unit = {
+    for (submission <- candidates) {
+      val driverCpu = submission.cores
+      val driverMem = submission.mem
+      logTrace(s"Finding offer to launch driver with cpu: $driverCpu, mem: $driverMem")
+      val offerOption = currentOffers.find { o =>
+        getResource(o.resources, "cpus") >= driverCpu &&
+        getResource(o.resources, "mem") >= driverMem
+      }
+      if (offerOption.isEmpty) {
+        logDebug(s"Unable to find offer to launch driver id: ${submission.submissionId}, " +
+          s"cpu: $driverCpu, mem: $driverMem")
+      } else {
+        val offer = offerOption.get
+        val queuedTasks = tasks.getOrElseUpdate(offer.offerId, new ArrayBuffer[TaskInfo])
+        val task = createTaskInfo(submission, offer)
+        queuedTasks += task
+        logTrace(s"Using offer ${offer.offerId.getValue} to launch driver " +
+          submission.submissionId)
+        val newState = new MesosClusterSubmissionState(submission, task.getTaskId, offer.slaveId,
+          None, new Date(), None, getDriverFrameworkID(submission))
+        launchedDrivers(submission.submissionId) = newState
+        launchedDriversState.persist(submission.submissionId, newState)
+        afterLaunchCallback(submission.submissionId)
+      }
+    }
+  }
+
+  override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = {
+    logTrace(s"Received offers from Mesos: \n${offers.asScala.mkString("\n")}")
+    val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]()
+    val currentTime = new Date()
+
+    val currentOffers = offers.asScala.map {
+      o => new ResourceOffer(o.getId, o.getSlaveId, o.getResourcesList)
+    }.toList
+
+    stateLock.synchronized {
+      // We first schedule all the supervised drivers that are ready to retry.
+      // This list will be empty if none of the drivers are marked as supervise.
+      val driversToRetry = pendingRetryDrivers.filter { d =>
+        d.retryState.get.nextRetry.before(currentTime)
+      }
+
+      scheduleTasks(
+        copyBuffer(driversToRetry),
+        removeFromPendingRetryDrivers,
+        currentOffers,
+        tasks)
+
+      // Then we walk through the queued drivers and try to schedule them.
+      scheduleTasks(
+        copyBuffer(queuedDrivers),
+        removeFromQueuedDrivers,
+        currentOffers,
+        tasks)
+    }
+    tasks.foreach { case (offerId, taskInfos) =>
+      driver.launchTasks(Collections.singleton(offerId), taskInfos.asJava)
+    }
+
+    for (o <- currentOffers if !tasks.contains(o.offerId)) {
+      driver.declineOffer(o.offerId)
+    }
+  }
+
+  private def copyBuffer(
+      buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = {
+    val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size)
+    buffer.copyToBuffer(newBuffer)
+    newBuffer
+  }
+
+  def getSchedulerState(): MesosClusterSchedulerState = {
+    stateLock.synchronized {
+      new MesosClusterSchedulerState(
+        frameworkId,
+        masterInfo.map(m => s"http://${m.getIp}:${m.getPort}"),
+        copyBuffer(queuedDrivers),
+        launchedDrivers.values.map(_.copy()).toList,
+        finishedDrivers.map(_.copy()).toList,
+        copyBuffer(pendingRetryDrivers))
+    }
+  }
+
+  override def offerRescinded(driver: SchedulerDriver, offerId: OfferID): Unit = {}
+  override def disconnected(driver: SchedulerDriver): Unit = {}
+  override def reregistered(driver: SchedulerDriver, masterInfo: MasterInfo): Unit = {
+    logInfo(s"Framework re-registered with master ${masterInfo.getId}")
+  }
+  override def slaveLost(driver: SchedulerDriver, slaveId: SlaveID): Unit = {}
+  override def error(driver: SchedulerDriver, error: String): Unit = {
+    logError("Error received: " + error)
+    markErr()
+  }
+
+  /**
+   * Check if the task state is a recoverable state that we can relaunch the task.
+   * Task state like TASK_ERROR are not relaunchable state since it wasn't able
+   * to be validated by Mesos.
+   */
+  private def shouldRelaunch(state: MesosTaskState): Boolean = {
+    state == MesosTaskState.TASK_FAILED ||
+      state == MesosTaskState.TASK_KILLED ||
+      state == MesosTaskState.TASK_LOST
+  }
+
+  override def statusUpdate(driver: SchedulerDriver, status: TaskStatus): Unit = {
+    val taskId = status.getTaskId.getValue
+    stateLock.synchronized {
+      if (launchedDrivers.contains(taskId)) {
+        if (status.getReason == Reason.REASON_RECONCILIATION &&
+          !pendingRecover.contains(taskId)) {
+          // Task has already received update and no longer requires reconciliation.
+          return
+        }
+        val state = launchedDrivers(taskId)
+        // Check if the driver is supervise enabled and can be relaunched.
+        if (state.driverDescription.supervise && shouldRelaunch(status.getState)) {
+          removeFromLaunchedDrivers(taskId)
+          state.finishDate = Some(new Date())
+          val retryState: Option[MesosClusterRetryState] = state.driverDescription.retryState
+          val (retries, waitTimeSec) = retryState
+            .map { rs => (rs.retries + 1, Math.min(maxRetryWaitTime, rs.waitTime * 2)) }
+            .getOrElse{ (1, 1) }
+          val nextRetry = new Date(new Date().getTime + waitTimeSec * 1000L)
+
+          val newDriverDescription = state.driverDescription.copy(
+            retryState = Some(new MesosClusterRetryState(status, retries, nextRetry, waitTimeSec)))
+          pendingRetryDrivers += newDriverDescription
+          pendingRetryDriversState.persist(taskId, newDriverDescription)
+        } else if (TaskState.isFinished(mesosToTaskState(status.getState))) {
+          removeFromLaunchedDrivers(taskId)
+          state.finishDate = Some(new Date())
+          if (finishedDrivers.size >= retainedDrivers) {
+            val toRemove = math.max(retainedDrivers / 10, 1)
+            finishedDrivers.trimStart(toRemove)
+          }
+          finishedDrivers += state
+        }
+        state.mesosTaskStatus = Option(status)
+      } else {
+        logError(s"Unable to find driver $taskId in status update")
+      }
+    }
+  }
+
+  override def frameworkMessage(
+      driver: SchedulerDriver,
+      executorId: ExecutorID,
+      slaveId: SlaveID,
+      message: Array[Byte]): Unit = {}
+
+  override def executorLost(
+      driver: SchedulerDriver,
+      executorId: ExecutorID,
+      slaveId: SlaveID,
+      status: Int): Unit = {}
+
+  private def removeFromQueuedDrivers(id: String): Boolean = {
+    val index = queuedDrivers.indexWhere(_.submissionId.equals(id))
+    if (index != -1) {
+      queuedDrivers.remove(index)
+      queuedDriversState.expunge(id)
+      true
+    } else {
+      false
+    }
+  }
+
+  private def removeFromLaunchedDrivers(id: String): Boolean = {
+    if (launchedDrivers.remove(id).isDefined) {
+      launchedDriversState.expunge(id)
+      true
+    } else {
+      false
+    }
+  }
+
+  private def removeFromPendingRetryDrivers(id: String): Boolean = {
+    val index = pendingRetryDrivers.indexWhere(_.submissionId.equals(id))
+    if (index != -1) {
+      pendingRetryDrivers.remove(index)
+      pendingRetryDriversState.expunge(id)
+      true
+    } else {
+      false
+    }
+  }
+
+  def getQueuedDriversSize: Int = queuedDrivers.size
+  def getLaunchedDriversSize: Int = launchedDrivers.size
+  def getPendingRetryDriversSize: Int = pendingRetryDrivers.size
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala
new file mode 100644
index 0000000..1fe9497
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.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.scheduler.cluster.mesos
+
+import com.codahale.metrics.{Gauge, MetricRegistry}
+
+import org.apache.spark.metrics.source.Source
+
+private[mesos] class MesosClusterSchedulerSource(scheduler: MesosClusterScheduler)
+  extends Source {
+  override def sourceName: String = "mesos_cluster"
+  override def metricRegistry: MetricRegistry = new MetricRegistry()
+
+  metricRegistry.register(MetricRegistry.name("waitingDrivers"), new Gauge[Int] {
+    override def getValue: Int = scheduler.getQueuedDriversSize
+  })
+
+  metricRegistry.register(MetricRegistry.name("launchedDrivers"), new Gauge[Int] {
+    override def getValue: Int = scheduler.getLaunchedDriversSize
+  })
+
+  metricRegistry.register(MetricRegistry.name("retryDrivers"), new Gauge[Int] {
+    override def getValue: Int = scheduler.getPendingRetryDriversSize
+  })
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
new file mode 100644
index 0000000..3258b09
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
@@ -0,0 +1,668 @@
+/*
+ * 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.mesos
+
+import java.io.File
+import java.util.{Collections, List => JList}
+import java.util.concurrent.locks.ReentrantLock
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.Future
+
+import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _}
+
+import org.apache.spark.{SecurityManager, SparkContext, SparkException, TaskState}
+import org.apache.spark.network.netty.SparkTransportConf
+import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient
+import org.apache.spark.rpc.RpcEndpointAddress
+import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.util.Utils
+
+/**
+ * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds
+ * onto each Mesos node for the duration of the Spark job instead of relinquishing cores whenever
+ * a task is done. It launches Spark tasks within the coarse-grained Mesos tasks using the
+ * CoarseGrainedSchedulerBackend mechanism. This class is useful for lower and more predictable
+ * latency.
+ *
+ * Unfortunately this has a bit of duplication from [[MesosFineGrainedSchedulerBackend]],
+ * but it seems hard to remove this.
+ */
+private[spark] class MesosCoarseGrainedSchedulerBackend(
+    scheduler: TaskSchedulerImpl,
+    sc: SparkContext,
+    master: String,
+    securityManager: SecurityManager)
+  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv)
+  with org.apache.mesos.Scheduler
+  with MesosSchedulerUtils {
+
+  val MAX_SLAVE_FAILURES = 2     // Blacklist a slave after this many failures
+
+  // Maximum number of cores to acquire (TODO: we'll need more flexible controls here)
+  val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt
+
+  val useFetcherCache = conf.getBoolean("spark.mesos.fetcherCache.enable", false)
+
+  val maxGpus = conf.getInt("spark.mesos.gpus.max", 0)
+
+  private[this] val shutdownTimeoutMS =
+    conf.getTimeAsMs("spark.mesos.coarse.shutdownTimeout", "10s")
+      .ensuring(_ >= 0, "spark.mesos.coarse.shutdownTimeout must be >= 0")
+
+  // Synchronization protected by stateLock
+  private[this] var stopCalled: Boolean = false
+
+  // If shuffle service is enabled, the Spark driver will register with the shuffle service.
+  // This is for cleaning up shuffle files reliably.
+  private val shuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false)
+
+  // Cores we have acquired with each Mesos task ID
+  val coresByTaskId = new mutable.HashMap[String, Int]
+  val gpusByTaskId = new mutable.HashMap[String, Int]
+  var totalCoresAcquired = 0
+  var totalGpusAcquired = 0
+
+  // SlaveID -> Slave
+  // This map accumulates entries for the duration of the job.  Slaves are never deleted, because
+  // we need to maintain e.g. failure state and connection state.
+  private val slaves = new mutable.HashMap[String, Slave]
+
+  /**
+   * The total number of executors we aim to have. Undefined when not using dynamic allocation.
+   * Initially set to 0 when using dynamic allocation, the executor allocation manager will send
+   * the real initial limit later.
+   */
+  private var executorLimitOption: Option[Int] = {
+    if (Utils.isDynamicAllocationEnabled(conf)) {
+      Some(0)
+    } else {
+      None
+    }
+  }
+
+  /**
+   *  Return the current executor limit, which may be [[Int.MaxValue]]
+   *  before properly initialized.
+   */
+  private[mesos] def executorLimit: Int = executorLimitOption.getOrElse(Int.MaxValue)
+
+  // private lock object protecting mutable state above. Using the intrinsic lock
+  // may lead to deadlocks since the superclass might also try to lock
+  private val stateLock = new ReentrantLock
+
+  val extraCoresPerExecutor = conf.getInt("spark.mesos.extra.cores", 0)
+
+  // Offer constraints
+  private val slaveOfferConstraints =
+    parseConstraintString(sc.conf.get("spark.mesos.constraints", ""))
+
+  // Reject offers with mismatched constraints in seconds
+  private val rejectOfferDurationForUnmetConstraints =
+    getRejectOfferDurationForUnmetConstraints(sc)
+
+  // Reject offers when we reached the maximum number of cores for this framework
+  private val rejectOfferDurationForReachedMaxCores =
+    getRejectOfferDurationForReachedMaxCores(sc)
+
+  // A client for talking to the external shuffle service
+  private val mesosExternalShuffleClient: Option[MesosExternalShuffleClient] = {
+    if (shuffleServiceEnabled) {
+      Some(getShuffleClient())
+    } else {
+      None
+    }
+  }
+
+  // This method is factored out for testability
+  protected def getShuffleClient(): MesosExternalShuffleClient = {
+    new MesosExternalShuffleClient(
+      SparkTransportConf.fromSparkConf(conf, "shuffle"),
+      securityManager,
+      securityManager.isAuthenticationEnabled(),
+      securityManager.isSaslEncryptionEnabled())
+  }
+
+  var nextMesosTaskId = 0
+
+  @volatile var appId: String = _
+
+  def newMesosTaskId(): String = {
+    val id = nextMesosTaskId
+    nextMesosTaskId += 1
+    id.toString
+  }
+
+  override def start() {
+    super.start()
+    val driver = createSchedulerDriver(
+      master,
+      MesosCoarseGrainedSchedulerBackend.this,
+      sc.sparkUser,
+      sc.appName,
+      sc.conf,
+      sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.webUrl)),
+      None,
+      None,
+      sc.conf.getOption("spark.mesos.driver.frameworkId")
+    )
+
+    unsetFrameworkID(sc)
+    startScheduler(driver)
+  }
+
+  def createCommand(offer: Offer, numCores: Int, taskId: String): CommandInfo = {
+    val environment = Environment.newBuilder()
+    val extraClassPath = conf.getOption("spark.executor.extraClassPath")
+    extraClassPath.foreach { cp =>
+      environment.addVariables(
+        Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build())
+    }
+    val extraJavaOpts = conf.get("spark.executor.extraJavaOptions", "")
+
+    // Set the environment variable through a command prefix
+    // to append to the existing value of the variable
+    val prefixEnv = conf.getOption("spark.executor.extraLibraryPath").map { p =>
+      Utils.libraryPathEnvPrefix(Seq(p))
+    }.getOrElse("")
+
+    environment.addVariables(
+      Environment.Variable.newBuilder()
+        .setName("SPARK_EXECUTOR_OPTS")
+        .setValue(extraJavaOpts)
+        .build())
+
+    sc.executorEnvs.foreach { case (key, value) =>
+      environment.addVariables(Environment.Variable.newBuilder()
+        .setName(key)
+        .setValue(value)
+        .build())
+    }
+    val command = CommandInfo.newBuilder()
+      .setEnvironment(environment)
+
+    val uri = conf.getOption("spark.executor.uri")
+      .orElse(Option(System.getenv("SPARK_EXECUTOR_URI")))
+
+    if (uri.isEmpty) {
+      val executorSparkHome = conf.getOption("spark.mesos.executor.home")
+        .orElse(sc.getSparkHome())
+        .getOrElse {
+          throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!")
+        }
+      val runScript = new File(executorSparkHome, "./bin/spark-class").getPath
+      command.setValue(
+        "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend"
+          .format(prefixEnv, runScript) +
+        s" --driver-url $driverURL" +
+        s" --executor-id $taskId" +
+        s" --hostname ${executorHostname(offer)}" +
+        s" --cores $numCores" +
+        s" --app-id $appId")
+    } else {
+      // Grab everything to the first '.'. We'll use that and '*' to
+      // glob the directory "correctly".
+      val basename = uri.get.split('/').last.split('.').head
+      command.setValue(
+        s"cd $basename*; $prefixEnv " +
+        "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" +
+        s" --driver-url $driverURL" +
+        s" --executor-id $taskId" +
+        s" --hostname ${executorHostname(offer)}" +
+        s" --cores $numCores" +
+        s" --app-id $appId")
+      command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get).setCache(useFetcherCache))
+    }
+
+    conf.getOption("spark.mesos.uris").foreach(setupUris(_, command, useFetcherCache))
+
+    command.build()
+  }
+
+  protected def driverURL: String = {
+    if (conf.contains("spark.testing")) {
+      "driverURL"
+    } else {
+      RpcEndpointAddress(
+        conf.get("spark.driver.host"),
+        conf.get("spark.driver.port").toInt,
+        CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+    }
+  }
+
+  override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID) {}
+
+  override def registered(
+      d: org.apache.mesos.SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) {
+    appId = frameworkId.getValue
+    mesosExternalShuffleClient.foreach(_.init(appId))
+    markRegistered()
+  }
+
+  override def sufficientResourcesRegistered(): Boolean = {
+    totalCoresAcquired >= maxCores * minRegisteredRatio
+  }
+
+  override def disconnected(d: org.apache.mesos.SchedulerDriver) {}
+
+  override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {}
+
+  /**
+   * Method called by Mesos to offer resources on slaves. We respond by launching an executor,
+   * unless we've already launched more than we wanted to.
+   */
+  override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]) {
+    stateLock.synchronized {
+      if (stopCalled) {
+        logDebug("Ignoring offers during shutdown")
+        // Driver should simply return a stopped status on race
+        // condition between this.stop() and completing here
+        offers.asScala.map(_.getId).foreach(d.declineOffer)
+        return
+      }
+
+      logDebug(s"Received ${offers.size} resource offers.")
+
+      val (matchedOffers, unmatchedOffers) = offers.asScala.partition { offer =>
+        val offerAttributes = toAttributeMap(offer.getAttributesList)
+        matchesAttributeRequirements(slaveOfferConstraints, offerAttributes)
+      }
+
+      declineUnmatchedOffers(d, unmatchedOffers)
+      handleMatchedOffers(d, matchedOffers)
+    }
+  }
+
+  private def declineUnmatchedOffers(
+      d: org.apache.mesos.SchedulerDriver, offers: mutable.Buffer[Offer]): Unit = {
+    offers.foreach { offer =>
+      declineOffer(d, offer, Some("unmet constraints"),
+        Some(rejectOfferDurationForUnmetConstraints))
+    }
+  }
+
+  private def declineOffer(
+      d: org.apache.mesos.SchedulerDriver,
+      offer: Offer,
+      reason: Option[String] = None,
+      refuseSeconds: Option[Long] = None): Unit = {
+
+    val id = offer.getId.getValue
+    val offerAttributes = toAttributeMap(offer.getAttributesList)
+    val mem = getResource(offer.getResourcesList, "mem")
+    val cpus = getResource(offer.getResourcesList, "cpus")
+    val ports = getRangeResource(offer.getResourcesList, "ports")
+
+    logDebug(s"Declining offer: $id with attributes: $offerAttributes mem: $mem" +
+      s" cpu: $cpus port: $ports for $refuseSeconds seconds" +
+      reason.map(r => s" (reason: $r)").getOrElse(""))
+
+    refuseSeconds match {
+      case Some(seconds) =>
+        val filters = Filters.newBuilder().setRefuseSeconds(seconds).build()
+        d.declineOffer(offer.getId, filters)
+      case _ => d.declineOffer(offer.getId)
+    }
+  }
+
+  /**
+   * Launches executors on accepted offers, and declines unused offers. Executors are launched
+   * round-robin on offers.
+   *
+   * @param d SchedulerDriver
+   * @param offers Mesos offers that match attribute constraints
+   */
+  private def handleMatchedOffers(
+      d: org.apache.mesos.SchedulerDriver, offers: mutable.Buffer[Offer]): Unit = {
+    val tasks = buildMesosTasks(offers)
+    for (offer <- offers) {
+      val offerAttributes = toAttributeMap(offer.getAttributesList)
+      val offerMem = getResource(offer.getResourcesList, "mem")
+      val offerCpus = getResource(offer.getResourcesList, "cpus")
+      val offerPorts = getRangeResource(offer.getResourcesList, "ports")
+      val id = offer.getId.getValue
+
+      if (tasks.contains(offer.getId)) { // accept
+        val offerTasks = tasks(offer.getId)
+
+        logDebug(s"Accepting offer: $id with attributes: $offerAttributes " +
+          s"mem: $offerMem cpu: $offerCpus ports: $offerPorts." +
+          s"  Launching ${offerTasks.size} Mesos tasks.")
+
+        for (task <- offerTasks) {
+          val taskId = task.getTaskId
+          val mem = getResource(task.getResourcesList, "mem")
+          val cpus = getResource(task.getResourcesList, "cpus")
+          val ports = getRangeResource(task.getResourcesList, "ports").mkString(",")
+
+          logDebug(s"Launching Mesos task: ${taskId.getValue} with mem: $mem cpu: $cpus" +
+            s" ports: $ports")
+        }
+
+        d.launchTasks(
+          Collections.singleton(offer.getId),
+          offerTasks.asJava)
+      } else if (totalCoresAcquired >= maxCores) {
+        // Reject an offer for a configurable amount of time to avoid starving other frameworks
+        declineOffer(d, offer, Some("reached spark.cores.max"),
+          Some(rejectOfferDurationForReachedMaxCores))
+      } else {
+        declineOffer(d, offer)
+      }
+    }
+  }
+
+  /**
+   * Returns a map from OfferIDs to the tasks to launch on those offers.  In order to maximize
+   * per-task memory and IO, tasks are round-robin assigned to offers.
+   *
+   * @param offers Mesos offers that match attribute constraints
+   * @return A map from OfferID to a list of Mesos tasks to launch on that offer
+   */
+  private def buildMesosTasks(offers: mutable.Buffer[Offer]): Map[OfferID, List[MesosTaskInfo]] = {
+    // offerID -> tasks
+    val tasks = new mutable.HashMap[OfferID, List[MesosTaskInfo]].withDefaultValue(Nil)
+
+    // offerID -> resources
+    val remainingResources = mutable.Map(offers.map(offer =>
+      (offer.getId.getValue, offer.getResourcesList)): _*)
+
+    var launchTasks = true
+
+    // TODO(mgummelt): combine offers for a single slave
+    //
+    // round-robin create executors on the available offers
+    while (launchTasks) {
+      launchTasks = false
+
+      for (offer <- offers) {
+        val slaveId = offer.getSlaveId.getValue
+        val offerId = offer.getId.getValue
+        val resources = remainingResources(offerId)
+
+        if (canLaunchTask(slaveId, resources)) {
+          // Create a task
+          launchTasks = true
+          val taskId = newMesosTaskId()
+          val offerCPUs = getResource(resources, "cpus").toInt
+          val taskGPUs = Math.min(
+            Math.max(0, maxGpus - totalGpusAcquired), getResource(resources, "gpus").toInt)
+
+          val taskCPUs = executorCores(offerCPUs)
+          val taskMemory = executorMemory(sc)
+
+          slaves.getOrElseUpdate(slaveId, new Slave(offer.getHostname)).taskIDs.add(taskId)
+
+          val (resourcesLeft, resourcesToUse) =
+            partitionTaskResources(resources, taskCPUs, taskMemory, taskGPUs)
+
+          val taskBuilder = MesosTaskInfo.newBuilder()
+            .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build())
+            .setSlaveId(offer.getSlaveId)
+            .setCommand(createCommand(offer, taskCPUs + extraCoresPerExecutor, taskId))
+            .setName("Task " + taskId)
+          taskBuilder.addAllResources(resourcesToUse.asJava)
+          taskBuilder.setContainer(MesosSchedulerBackendUtil.containerInfo(sc.conf))
+
+          tasks(offer.getId) ::= taskBuilder.build()
+          remainingResources(offerId) = resourcesLeft.asJava
+          totalCoresAcquired += taskCPUs
+          coresByTaskId(taskId) = taskCPUs
+          if (taskGPUs > 0) {
+            totalGpusAcquired += taskGPUs
+            gpusByTaskId(taskId) = taskGPUs
+          }
+        }
+      }
+    }
+    tasks.toMap
+  }
+
+  /** Extracts task needed resources from a list of available resources. */
+  private def partitionTaskResources(
+      resources: JList[Resource],
+      taskCPUs: Int,
+      taskMemory: Int,
+      taskGPUs: Int)
+    : (List[Resource], List[Resource]) = {
+
+    // partition cpus & mem
+    val (afterCPUResources, cpuResourcesToUse) = partitionResources(resources, "cpus", taskCPUs)
+    val (afterMemResources, memResourcesToUse) =
+      partitionResources(afterCPUResources.asJava, "mem", taskMemory)
+    val (afterGPUResources, gpuResourcesToUse) =
+      partitionResources(afterMemResources.asJava, "gpus", taskGPUs)
+
+    // If user specifies port numbers in SparkConfig then consecutive tasks will not be launched
+    // on the same host. This essentially means one executor per host.
+    // TODO: handle network isolator case
+    val (nonPortResources, portResourcesToUse) =
+      partitionPortResources(nonZeroPortValuesFromConfig(sc.conf), afterGPUResources)
+
+    (nonPortResources,
+      cpuResourcesToUse ++ memResourcesToUse ++ portResourcesToUse ++ gpuResourcesToUse)
+  }
+
+  private def canLaunchTask(slaveId: String, resources: JList[Resource]): Boolean = {
+    val offerMem = getResource(resources, "mem")
+    val offerCPUs = getResource(resources, "cpus").toInt
+    val cpus = executorCores(offerCPUs)
+    val mem = executorMemory(sc)
+    val ports = getRangeResource(resources, "ports")
+    val meetsPortRequirements = checkPorts(sc.conf, ports)
+
+    cpus > 0 &&
+      cpus <= offerCPUs &&
+      cpus + totalCoresAcquired <= maxCores &&
+      mem <= offerMem &&
+      numExecutors() < executorLimit &&
+      slaves.get(slaveId).map(_.taskFailures).getOrElse(0) < MAX_SLAVE_FAILURES &&
+      meetsPortRequirements
+  }
+
+  private def executorCores(offerCPUs: Int): Int = {
+    sc.conf.getInt("spark.executor.cores",
+      math.min(offerCPUs, maxCores - totalCoresAcquired))
+  }
+
+  override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus) {
+    val taskId = status.getTaskId.getValue
+    val slaveId = status.getSlaveId.getValue
+    val state = mesosToTaskState(status.getState)
+
+    logInfo(s"Mesos task $taskId is now ${status.getState}")
+
+    stateLock.synchronized {
+      val slave = slaves(slaveId)
+
+      // If the shuffle service is enabled, have the driver register with each one of the
+      // shuffle services. This allows the shuffle services to clean up state associated with
+      // this application when the driver exits. There is currently not a great way to detect
+      // this through Mesos, since the shuffle services are set up independently.
+      if (state.equals(TaskState.RUNNING) &&
+          shuffleServiceEnabled &&
+          !slave.shuffleRegistered) {
+        assume(mesosExternalShuffleClient.isDefined,
+          "External shuffle client was not instantiated even though shuffle service is enabled.")
+        // TODO: Remove this and allow the MesosExternalShuffleService to detect
+        // framework termination when new Mesos Framework HTTP API is available.
+        val externalShufflePort = conf.getInt("spark.shuffle.service.port", 7337)
+
+        logDebug(s"Connecting to shuffle service on slave $slaveId, " +
+            s"host ${slave.hostname}, port $externalShufflePort for app ${conf.getAppId}")
+
+        mesosExternalShuffleClient.get
+          .registerDriverWithShuffleService(
+            slave.hostname,
+            externalShufflePort,
+            sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs",
+              s"${sc.conf.getTimeAsMs("spark.network.timeout", "120s")}ms"),
+            sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s"))
+        slave.shuffleRegistered = true
+      }
+
+      if (TaskState.isFinished(state)) {
+        // Remove the cores we have remembered for this task, if it's in the hashmap
+        for (cores <- coresByTaskId.get(taskId)) {
+          totalCoresAcquired -= cores
+          coresByTaskId -= taskId
+        }
+        // Also remove the gpus we have remembered for this task, if it's in the hashmap
+        for (gpus <- gpusByTaskId.get(taskId)) {
+          totalGpusAcquired -= gpus
+          gpusByTaskId -= taskId
+        }
+        // If it was a failure, mark the slave as failed for blacklisting purposes
+        if (TaskState.isFailed(state)) {
+          slave.taskFailures += 1
+
+          if (slave.taskFailures >= MAX_SLAVE_FAILURES) {
+            logInfo(s"Blacklisting Mesos slave $slaveId due to too many failures; " +
+                "is Spark installed on it?")
+          }
+        }
+        executorTerminated(d, slaveId, taskId, s"Executor finished with state $state")
+        // In case we'd rejected everything before but have now lost a node
+        d.reviveOffers()
+      }
+    }
+  }
+
+  override def error(d: org.apache.mesos.SchedulerDriver, message: String) {
+    logError(s"Mesos error: $message")
+    scheduler.error(message)
+  }
+
+  override def stop() {
+    // Make sure we're not launching tasks during shutdown
+    stateLock.synchronized {
+      if (stopCalled) {
+        logWarning("Stop called multiple times, ignoring")
+        return
+      }
+      stopCalled = true
+      super.stop()
+    }
+
+    // Wait for executors to report done, or else mesosDriver.stop() will forcefully kill them.
+    // See SPARK-12330
+    val startTime = System.nanoTime()
+
+    // slaveIdsWithExecutors has no memory barrier, so this is eventually consistent
+    while (numExecutors() > 0 &&
+      System.nanoTime() - startTime < shutdownTimeoutMS * 1000L * 1000L) {
+      Thread.sleep(100)
+    }
+
+    if (numExecutors() > 0) {
+      logWarning(s"Timed out waiting for ${numExecutors()} remaining executors "
+        + s"to terminate within $shutdownTimeoutMS ms. This may leave temporary files "
+        + "on the mesos nodes.")
+    }
+
+    // Close the mesos external shuffle client if used
+    mesosExternalShuffleClient.foreach(_.close())
+
+    if (mesosDriver != null) {
+      mesosDriver.stop()
+    }
+  }
+
+  override def frameworkMessage(
+      d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {}
+
+  /**
+   * Called when a slave is lost or a Mesos task finished. Updates local view on
+   * what tasks are running. It also notifies the driver that an executor was removed.
+   */
+  private def executorTerminated(
+      d: org.apache.mesos.SchedulerDriver,
+      slaveId: String,
+      taskId: String,
+      reason: String): Unit = {
+    stateLock.synchronized {
+      // Do not call removeExecutor() after this scheduler backend was stopped because
+      // removeExecutor() internally will send a message to the driver endpoint but
+      // the driver endpoint is not available now, otherwise an exception will be thrown.
+      if (!stopCalled) {
+        removeExecutor(taskId, SlaveLost(reason))
+      }
+      slaves(slaveId).taskIDs.remove(taskId)
+    }
+  }
+
+  override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID): Unit = {
+    logInfo(s"Mesos slave lost: ${slaveId.getValue}")
+  }
+
+  override def executorLost(
+      d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int): Unit = {
+    logInfo("Mesos executor lost: %s".format(e.getValue))
+  }
+
+  override def applicationId(): String =
+    Option(appId).getOrElse {
+      logWarning("Application ID is not initialized yet.")
+      super.applicationId
+    }
+
+  override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future.successful {
+    // We don't truly know if we can fulfill the full amount of executors
+    // since at coarse grain it depends on the amount of slaves available.
+    logInfo("Capping the total amount of executors to " + requestedTotal)
+    executorLimitOption = Some(requestedTotal)
+    true
+  }
+
+  override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future.successful {
+    if (mesosDriver == null) {
+      logWarning("Asked to kill executors before the Mesos driver was started.")
+      false
+    } else {
+      for (executorId <- executorIds) {
+        val taskId = TaskID.newBuilder().setValue(executorId).build()
+        mesosDriver.killTask(taskId)
+      }
+      // no need to adjust `executorLimitOption` since the AllocationManager already communicated
+      // the desired limit through a call to `doRequestTotalExecutors`.
+      // See [[o.a.s.scheduler.cluster.CoarseGrainedSchedulerBackend.killExecutors]]
+      true
+    }
+  }
+
+  private def numExecutors(): Int = {
+    slaves.values.map(_.taskIDs.size).sum
+  }
+
+  private def executorHostname(offer: Offer): String = {
+    if (sc.conf.getOption("spark.mesos.network.name").isDefined) {
+      // The agent's IP is not visible in a CNI container, so we bind to 0.0.0.0
+      "0.0.0.0"
+    } else {
+      offer.getHostname
+    }
+  }
+}
+
+private class Slave(val hostname: String) {
+  val taskIDs = new mutable.HashSet[String]()
+  var taskFailures = 0
+  var shuffleRegistered = false
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala
new file mode 100644
index 0000000..779ffb5
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala
@@ -0,0 +1,444 @@
+/*
+ * 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.mesos
+
+import java.io.File
+import java.util.{ArrayList => JArrayList, Collections, List => JList}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{HashMap, HashSet}
+
+import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, TaskInfo => MesosTaskInfo, _}
+import org.apache.mesos.protobuf.ByteString
+
+import org.apache.spark.{SparkContext, SparkException, TaskState}
+import org.apache.spark.executor.MesosExecutorBackend
+import org.apache.spark.scheduler._
+import org.apache.spark.scheduler.cluster.ExecutorInfo
+import org.apache.spark.util.Utils
+
+/**
+ * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a
+ * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks
+ * from multiple apps can run on different cores) and in time (a core can switch ownership).
+ */
+private[spark] class MesosFineGrainedSchedulerBackend(
+    scheduler: TaskSchedulerImpl,
+    sc: SparkContext,
+    master: String)
+  extends SchedulerBackend
+  with org.apache.mesos.Scheduler
+  with MesosSchedulerUtils {
+
+  // Stores the slave ids that has launched a Mesos executor.
+  val slaveIdToExecutorInfo = new HashMap[String, MesosExecutorInfo]
+  val taskIdToSlaveId = new HashMap[Long, String]
+
+  // An ExecutorInfo for our tasks
+  var execArgs: Array[Byte] = null
+
+  var classLoader: ClassLoader = null
+
+  // The listener bus to publish executor added/removed events.
+  val listenerBus = sc.listenerBus
+
+  private[mesos] val mesosExecutorCores = sc.conf.getDouble("spark.mesos.mesosExecutor.cores", 1)
+
+  // Offer constraints
+  private[this] val slaveOfferConstraints =
+    parseConstraintString(sc.conf.get("spark.mesos.constraints", ""))
+
+  // reject offers with mismatched constraints in seconds
+  private val rejectOfferDurationForUnmetConstraints =
+    getRejectOfferDurationForUnmetConstraints(sc)
+
+  @volatile var appId: String = _
+
+  override def start() {
+    classLoader = Thread.currentThread.getContextClassLoader
+    val driver = createSchedulerDriver(
+      master,
+      MesosFineGrainedSchedulerBackend.this,
+      sc.sparkUser,
+      sc.appName,
+      sc.conf,
+      sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.webUrl)),
+      Option.empty,
+      Option.empty,
+      sc.conf.getOption("spark.mesos.driver.frameworkId")
+    )
+
+    unsetFrameworkID(sc)
+    startScheduler(driver)
+  }
+
+  /**
+   * Creates a MesosExecutorInfo that is used to launch a Mesos executor.
+   * @param availableResources Available resources that is offered by Mesos
+   * @param execId The executor id to assign to this new executor.
+   * @return A tuple of the new mesos executor info and the remaining available resources.
+   */
+  def createExecutorInfo(
+      availableResources: JList[Resource],
+      execId: String): (MesosExecutorInfo, JList[Resource]) = {
+    val executorSparkHome = sc.conf.getOption("spark.mesos.executor.home")
+      .orElse(sc.getSparkHome()) // Fall back to driver Spark home for backward compatibility
+      .getOrElse {
+      throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!")
+    }
+    val environment = Environment.newBuilder()
+    sc.conf.getOption("spark.executor.extraClassPath").foreach { cp =>
+      environment.addVariables(
+        Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build())
+    }
+    val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions").getOrElse("")
+
+    val prefixEnv = sc.conf.getOption("spark.executor.extraLibraryPath").map { p =>
+      Utils.libraryPathEnvPrefix(Seq(p))
+    }.getOrElse("")
+
+    environment.addVariables(
+      Environment.Variable.newBuilder()
+        .setName("SPARK_EXECUTOR_OPTS")
+        .setValue(extraJavaOpts)
+        .build())
+    sc.executorEnvs.foreach { case (key, value) =>
+      environment.addVariables(Environment.Variable.newBuilder()
+        .setName(key)
+        .setValue(value)
+        .build())
+    }
+    val command = CommandInfo.newBuilder()
+      .setEnvironment(environment)
+    val uri = sc.conf.getOption("spark.executor.uri")
+      .orElse(Option(System.getenv("SPARK_EXECUTOR_URI")))
+
+    val executorBackendName = classOf[MesosExecutorBackend].getName
+    if (uri.isEmpty) {
+      val executorPath = new File(executorSparkHome, "/bin/spark-class").getPath
+      command.setValue(s"$prefixEnv $executorPath $executorBackendName")
+    } else {
+      // Grab everything to the first '.'. We'll use that and '*' to
+      // glob the directory "correctly".
+      val basename = uri.get.split('/').last.split('.').head
+      command.setValue(s"cd ${basename}*; $prefixEnv ./bin/spark-class $executorBackendName")
+      command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get))
+    }
+    val builder = MesosExecutorInfo.newBuilder()
+    val (resourcesAfterCpu, usedCpuResources) =
+      partitionResources(availableResources, "cpus", mesosExecutorCores)
+    val (resourcesAfterMem, usedMemResources) =
+      partitionResources(resourcesAfterCpu.asJava, "mem", executorMemory(sc))
+
+    builder.addAllResources(usedCpuResources.asJava)
+    builder.addAllResources(usedMemResources.asJava)
+
+    sc.conf.getOption("spark.mesos.uris").foreach(setupUris(_, command))
+
+    val executorInfo = builder
+      .setExecutorId(ExecutorID.newBuilder().setValue(execId).build())
+      .setCommand(command)
+      .setData(ByteString.copyFrom(createExecArg()))
+
+    executorInfo.setContainer(MesosSchedulerBackendUtil.containerInfo(sc.conf))
+    (executorInfo.build(), resourcesAfterMem.asJava)
+  }
+
+  /**
+   * Create and serialize the executor argument to pass to Mesos. Our executor arg is an array
+   * containing all the spark.* system properties in the form of (String, String) pairs.
+   */
+  private def createExecArg(): Array[Byte] = {
+    if (execArgs == null) {
+      val props = new HashMap[String, String]
+      for ((key, value) <- sc.conf.getAll) {
+        props(key) = value
+      }
+      // Serialize the map as an array of (String, String) pairs
+      execArgs = Utils.serialize(props.toArray)
+    }
+    execArgs
+  }
+
+  override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID) {}
+
+  override def registered(
+      d: org.apache.mesos.SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) {
+    inClassLoader() {
+      appId = frameworkId.getValue
+      logInfo("Registered as framework ID " + appId)
+      markRegistered()
+    }
+  }
+
+  private def inClassLoader()(fun: => Unit) = {
+    val oldClassLoader = Thread.currentThread.getContextClassLoader
+    Thread.currentThread.setContextClassLoader(classLoader)
+    try {
+      fun
+    } finally {
+      Thread.currentThread.setContextClassLoader(oldClassLoader)
+    }
+  }
+
+  override def disconnected(d: org.apache.mesos.SchedulerDriver) {}
+
+  override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {}
+
+  private def getTasksSummary(tasks: JArrayList[MesosTaskInfo]): String = {
+    val builder = new StringBuilder
+    tasks.asScala.foreach { t =>
+      builder.append("Task id: ").append(t.getTaskId.getValue).append("\n")
+        .append("Slave id: ").append(t.getSlaveId.getValue).append("\n")
+        .append("Task resources: ").append(t.getResourcesList).append("\n")
+        .append("Executor resources: ").append(t.getExecutor.getResourcesList)
+        .append("---------------------------------------------\n")
+    }
+    builder.toString()
+  }
+
+  /**
+   * Method called by Mesos to offer resources on slaves. We respond by asking our active task sets
+   * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that
+   * tasks are balanced across the cluster.
+   */
+  override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]) {
+    inClassLoader() {
+      // Fail first on offers with unmet constraints
+      val (offersMatchingConstraints, offersNotMatchingConstraints) =
+        offers.asScala.partition { o =>
+          val offerAttributes = toAttributeMap(o.getAttributesList)
+          val meetsConstraints =
+            matchesAttributeRequirements(slaveOfferConstraints, offerAttributes)
+
+          // add some debug messaging
+          if (!meetsConstraints) {
+            val id = o.getId.getValue
+            logDebug(s"Declining offer: $id with attributes: $offerAttributes")
+          }
+
+          meetsConstraints
+        }
+
+      // These offers do not meet constraints. We don't need to see them again.
+      // Decline the offer for a long period of time.
+      offersNotMatchingConstraints.foreach { o =>
+        d.declineOffer(o.getId, Filters.newBuilder()
+          .setRefuseSeconds(rejectOfferDurationForUnmetConstraints).build())
+      }
+
+      // Of the matching constraints, see which ones give us enough memory and cores
+      val (usableOffers, unUsableOffers) = offersMatchingConstraints.partition { o =>
+        val mem = getResource(o.getResourcesList, "mem")
+        val cpus = getResource(o.getResourcesList, "cpus")
+        val slaveId = o.getSlaveId.getValue
+        val offerAttributes = toAttributeMap(o.getAttributesList)
+
+        // check offers for
+        //  1. Memory requirements
+        //  2. CPU requirements - need at least 1 for executor, 1 for task
+        val meetsMemoryRequirements = mem >= executorMemory(sc)
+        val meetsCPURequirements = cpus >= (mesosExecutorCores + scheduler.CPUS_PER_TASK)
+        val meetsRequirements =
+          (meetsMemoryRequirements && meetsCPURequirements) ||
+          (slaveIdToExecutorInfo.contains(slaveId) && cpus >= scheduler.CPUS_PER_TASK)
+        val debugstr = if (meetsRequirements) "Accepting" else "Declining"
+        logDebug(s"$debugstr offer: ${o.getId.getValue} with attributes: "
+          + s"$offerAttributes mem: $mem cpu: $cpus")
+
+        meetsRequirements
+      }
+
+      // Decline offers we ruled out immediately
+      unUsableOffers.foreach(o => d.declineOffer(o.getId))
+
+      val workerOffers = usableOffers.map { o =>
+        val cpus = if (slaveIdToExecutorInfo.contains(o.getSlaveId.getValue)) {
+          getResource(o.getResourcesList, "cpus").toInt
+        } else {
+          // If the Mesos executor has not been started on this slave yet, set aside a few
+          // cores for the Mesos executor by offering fewer cores to the Spark executor
+          (getResource(o.getResourcesList, "cpus") - mesosExecutorCores).toInt
+        }
+        new WorkerOffer(
+          o.getSlaveId.getValue,
+          o.getHostname,
+          cpus)
+      }.toIndexedSeq
+
+      val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap
+      val slaveIdToWorkerOffer = workerOffers.map(o => o.executorId -> o).toMap
+      val slaveIdToResources = new HashMap[String, JList[Resource]]()
+      usableOffers.foreach { o =>
+        slaveIdToResources(o.getSlaveId.getValue) = o.getResourcesList
+      }
+
+      val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]]
+
+      val slavesIdsOfAcceptedOffers = HashSet[String]()
+
+      // Call into the TaskSchedulerImpl
+      val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty)
+      acceptedOffers
+        .foreach { offer =>
+          offer.foreach { taskDesc =>
+            val slaveId = taskDesc.executorId
+            slavesIdsOfAcceptedOffers += slaveId
+            taskIdToSlaveId(taskDesc.taskId) = slaveId
+            val (mesosTask, remainingResources) = createMesosTask(
+              taskDesc,
+              slaveIdToResources(slaveId),
+              slaveId)
+            mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo])
+              .add(mesosTask)
+            slaveIdToResources(slaveId) = remainingResources
+          }
+        }
+
+      // Reply to the offers
+      val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout?
+
+      mesosTasks.foreach { case (slaveId, tasks) =>
+        slaveIdToWorkerOffer.get(slaveId).foreach(o =>
+          listenerBus.post(SparkListenerExecutorAdded(System.currentTimeMillis(), slaveId,
+            // TODO: Add support for log urls for Mesos
+            new ExecutorInfo(o.host, o.cores, Map.empty)))
+        )
+        logTrace(s"Launching Mesos tasks on slave '$slaveId', tasks:\n${getTasksSummary(tasks)}")
+        d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters)
+      }
+
+      // Decline offers that weren't used
+      // NOTE: This logic assumes that we only get a single offer for each host in a given batch
+      for (o <- usableOffers if !slavesIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) {
+        d.declineOffer(o.getId)
+      }
+    }
+  }
+
+  /** Turn a Spark TaskDescription into a Mesos task and also resources unused by the task */
+  def createMesosTask(
+      task: TaskDescription,
+      resources: JList[Resource],
+      slaveId: String): (MesosTaskInfo, JList[Resource]) = {
+    val taskId = TaskID.newBuilder().setValue(task.taskId.toString).build()
+    val (executorInfo, remainingResources) = if (slaveIdToExecutorInfo.contains(slaveId)) {
+      (slaveIdToExecutorInfo(slaveId), resources)
+    } else {
+      createExecutorInfo(resources, slaveId)
+    }
+    slaveIdToExecutorInfo(slaveId) = executorInfo
+    val (finalResources, cpuResources) =
+      partitionResources(remainingResources, "cpus", scheduler.CPUS_PER_TASK)
+    val taskInfo = MesosTaskInfo.newBuilder()
+      .setTaskId(taskId)
+      .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build())
+      .setExecutor(executorInfo)
+      .setName(task.name)
+      .addAllResources(cpuResources.asJava)
+      .setData(MesosTaskLaunchData(task.serializedTask, task.attemptNumber).toByteString)
+      .build()
+    (taskInfo, finalResources.asJava)
+  }
+
+  override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus) {
+    inClassLoader() {
+      val tid = status.getTaskId.getValue.toLong
+      val state = mesosToTaskState(status.getState)
+      synchronized {
+        if (TaskState.isFailed(mesosToTaskState(status.getState))
+          && taskIdToSlaveId.contains(tid)) {
+          // We lost the executor on this slave, so remember that it's gone
+          removeExecutor(taskIdToSlaveId(tid), "Lost executor")
+        }
+        if (TaskState.isFinished(state)) {
+          taskIdToSlaveId.remove(tid)
+        }
+      }
+      scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer)
+    }
+  }
+
+  override def error(d: org.apache.mesos.SchedulerDriver, message: String) {
+    inClassLoader() {
+      logError("Mesos error: " + message)
+      markErr()
+      scheduler.error(message)
+    }
+  }
+
+  override def stop() {
+    if (mesosDriver != null) {
+      mesosDriver.stop()
+    }
+  }
+
+  override def reviveOffers() {
+    mesosDriver.reviveOffers()
+  }
+
+  override def frameworkMessage(
+      d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {}
+
+  /**
+   * Remove executor associated with slaveId in a thread safe manner.
+   */
+  private def removeExecutor(slaveId: String, reason: String) = {
+    synchronized {
+      listenerBus.post(SparkListenerExecutorRemoved(System.currentTimeMillis(), slaveId, reason))
+      slaveIdToExecutorInfo -= slaveId
+    }
+  }
+
+  private def recordSlaveLost(
+      d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) {
+    inClassLoader() {
+      logInfo("Mesos slave lost: " + slaveId.getValue)
+      removeExecutor(slaveId.getValue, reason.toString)
+      scheduler.executorLost(slaveId.getValue, reason)
+    }
+  }
+
+  override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID) {
+    recordSlaveLost(d, slaveId, SlaveLost())
+  }
+
+  override def executorLost(
+      d: org.apache.mesos.SchedulerDriver, executorId: ExecutorID, slaveId: SlaveID, status: Int) {
+    logInfo("Executor lost: %s, marking slave %s as lost".format(executorId.getValue,
+                                                                 slaveId.getValue))
+    recordSlaveLost(d, slaveId, ExecutorExited(status, exitCausedByApp = true))
+  }
+
+  override def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = {
+    mesosDriver.killTask(
+      TaskID.newBuilder()
+        .setValue(taskId.toString).build()
+    )
+  }
+
+  // TODO: query Mesos for number of cores
+  override def defaultParallelism(): Int = sc.conf.getInt("spark.default.parallelism", 8)
+
+  override def applicationId(): String =
+    Option(appId).getOrElse {
+      logWarning("Application ID is not initialized yet.")
+      super.applicationId
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala
new file mode 100644
index 0000000..a2adb22
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala
@@ -0,0 +1,165 @@
+/*
+ * 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.mesos
+
+import org.apache.mesos.Protos.{ContainerInfo, Image, NetworkInfo, Volume}
+import org.apache.mesos.Protos.ContainerInfo.{DockerInfo, MesosInfo}
+
+import org.apache.spark.{SparkConf, SparkException}
+import org.apache.spark.internal.Logging
+
+/**
+ * A collection of utility functions which can be used by both the
+ * MesosSchedulerBackend and the [[MesosFineGrainedSchedulerBackend]].
+ */
+private[mesos] object MesosSchedulerBackendUtil extends Logging {
+  /**
+   * Parse a comma-delimited list of volume specs, each of which
+   * takes the form [host-dir:]container-dir[:rw|:ro].
+   */
+  def parseVolumesSpec(volumes: String): List[Volume] = {
+    volumes.split(",").map(_.split(":")).flatMap { spec =>
+        val vol: Volume.Builder = Volume
+          .newBuilder()
+          .setMode(Volume.Mode.RW)
+        spec match {
+          case Array(container_path) =>
+            Some(vol.setContainerPath(container_path))
+          case Array(container_path, "rw") =>
+            Some(vol.setContainerPath(container_path))
+          case Array(container_path, "ro") =>
+            Some(vol.setContainerPath(container_path)
+              .setMode(Volume.Mode.RO))
+          case Array(host_path, container_path) =>
+            Some(vol.setContainerPath(container_path)
+              .setHostPath(host_path))
+          case Array(host_path, container_path, "rw") =>
+            Some(vol.setContainerPath(container_path)
+              .setHostPath(host_path))
+          case Array(host_path, container_path, "ro") =>
+            Some(vol.setContainerPath(container_path)
+              .setHostPath(host_path)
+              .setMode(Volume.Mode.RO))
+          case spec =>
+            logWarning(s"Unable to parse volume specs: $volumes. "
+              + "Expected form: \"[host-dir:]container-dir[:rw|:ro](, ...)\"")
+            None
+      }
+    }
+    .map { _.build() }
+    .toList
+  }
+
+  /**
+   * Parse a comma-delimited list of port mapping specs, each of which
+   * takes the form host_port:container_port[:udp|:tcp]
+   *
+   * Note:
+   * the docker form is [ip:]host_port:container_port, but the DockerInfo
+   * message has no field for 'ip', and instead has a 'protocol' field.
+   * Docker itself only appears to support TCP, so this alternative form
+   * anticipates the expansion of the docker form to allow for a protocol
+   * and leaves open the chance for mesos to begin to accept an 'ip' field
+   */
+  def parsePortMappingsSpec(portmaps: String): List[DockerInfo.PortMapping] = {
+    portmaps.split(",").map(_.split(":")).flatMap { spec: Array[String] =>
+      val portmap: DockerInfo.PortMapping.Builder = DockerInfo.PortMapping
+        .newBuilder()
+        .setProtocol("tcp")
+      spec match {
+        case Array(host_port, container_port) =>
+          Some(portmap.setHostPort(host_port.toInt)
+            .setContainerPort(container_port.toInt))
+        case Array(host_port, container_port, protocol) =>
+          Some(portmap.setHostPort(host_port.toInt)
+            .setContainerPort(container_port.toInt)
+            .setProtocol(protocol))
+        case spec =>
+          logWarning(s"Unable to parse port mapping specs: $portmaps. "
+            + "Expected form: \"host_port:container_port[:udp|:tcp](, ...)\"")
+          None
+      }
+    }
+    .map { _.build() }
+    .toList
+  }
+
+  def containerInfo(conf: SparkConf): ContainerInfo = {
+    val containerType = if (conf.contains("spark.mesos.executor.docker.image") &&
+      conf.get("spark.mesos.containerizer", "docker") == "docker") {
+      ContainerInfo.Type.DOCKER
+    } else {
+      ContainerInfo.Type.MESOS
+    }
+
+    val containerInfo = ContainerInfo.newBuilder()
+      .setType(containerType)
+
+    conf.getOption("spark.mesos.executor.docker.image").map { image =>
+      val forcePullImage = conf
+        .getOption("spark.mesos.executor.docker.forcePullImage")
+        .exists(_.equals("true"))
+
+      val portMaps = conf
+        .getOption("spark.mesos.executor.docker.portmaps")
+        .map(parsePortMappingsSpec)
+        .getOrElse(List.empty)
+
+      if (containerType == ContainerInfo.Type.DOCKER) {
+        containerInfo.setDocker(dockerInfo(image, forcePullImage, portMaps))
+      } else {
+        containerInfo.setMesos(mesosInfo(image, forcePullImage))
+      }
+
+      val volumes = conf
+        .getOption("spark.mesos.executor.docker.volumes")
+        .map(parseVolumesSpec)
+
+      volumes.foreach(_.foreach(containerInfo.addVolumes(_)))
+    }
+
+    conf.getOption("spark.mesos.network.name").map { name =>
+      val info = NetworkInfo.newBuilder().setName(name).build()
+      containerInfo.addNetworkInfos(info)
+    }
+
+    containerInfo.build()
+  }
+
+  private def dockerInfo(
+      image: String,
+      forcePullImage: Boolean,
+      portMaps: List[ContainerInfo.DockerInfo.PortMapping]): DockerInfo = {
+    val dockerBuilder = ContainerInfo.DockerInfo.newBuilder()
+      .setImage(image)
+      .setForcePullImage(forcePullImage)
+    portMaps.foreach(dockerBuilder.addPortMappings(_))
+
+    dockerBuilder.build
+  }
+
+  private def mesosInfo(image: String, forcePullImage: Boolean): MesosInfo = {
+    val imageProto = Image.newBuilder()
+      .setType(Image.Type.DOCKER)
+      .setDocker(Image.Docker.newBuilder().setName(image))
+      .setCached(!forcePullImage)
+    ContainerInfo.MesosInfo.newBuilder()
+      .setImage(imageProto)
+      .build
+  }
+}


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


[10/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..8772e26
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
@@ -0,0 +1,224 @@
+/*
+ * 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 scala.collection.mutable.{ArrayBuffer, HashMap, Set}
+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._
+
+private[yarn] case class ContainerLocalityPreferences(nodes: Array[String], racks: Array[String])
+
+/**
+ * This strategy is calculating the optimal locality preferences of YARN containers by considering
+ * the node ratio of pending tasks, number of required cores/containers and and locality of current
+ * existing and pending allocated containers. The target of this algorithm is to maximize the number
+ * of tasks that would run locally.
+ *
+ * Consider a situation in which we have 20 tasks that require (host1, host2, host3)
+ * and 10 tasks that require (host1, host2, host4), besides each container has 2 cores
+ * and cpus per task is 1, so the required container number is 15,
+ * and host ratio is (host1: 30, host2: 30, host3: 20, host4: 10).
+ *
+ * 1. If requested container number (18) is more than the required container number (15):
+ *
+ * requests for 5 containers with nodes: (host1, host2, host3, host4)
+ * requests for 5 containers with nodes: (host1, host2, host3)
+ * requests for 5 containers with nodes: (host1, host2)
+ * requests for 3 containers with no locality preferences.
+ *
+ * The placement ratio is 3 : 3 : 2 : 1, and set the additional containers with no locality
+ * preferences.
+ *
+ * 2. If requested container number (10) is less than or equal to the required container number
+ * (15):
+ *
+ * requests for 4 containers with nodes: (host1, host2, host3, host4)
+ * requests for 3 containers with nodes: (host1, host2, host3)
+ * requests for 3 containers with nodes: (host1, host2)
+ *
+ * The placement ratio is 10 : 10 : 7 : 4, close to expected ratio (3 : 3 : 2 : 1)
+ *
+ * 3. If containers exist but none of them can match the requested localities,
+ * follow the method of 1 and 2.
+ *
+ * 4. If containers exist and some of them can match the requested localities.
+ * For example if we have 1 containers on each node (host1: 1, host2: 1: host3: 1, host4: 1),
+ * and the expected containers on each node would be (host1: 5, host2: 5, host3: 4, host4: 2),
+ * so the newly requested containers on each node would be updated to (host1: 4, host2: 4,
+ * host3: 3, host4: 1), 12 containers by total.
+ *
+ *   4.1 If requested container number (18) is more than newly required containers (12). Follow
+ *   method 1 with updated ratio 4 : 4 : 3 : 1.
+ *
+ *   4.2 If request container number (10) is more than newly required containers (12). Follow
+ *   method 2 with updated ratio 4 : 4 : 3 : 1.
+ *
+ * 5. If containers exist and existing localities can fully cover the requested localities.
+ * For example if we have 5 containers on each node (host1: 5, host2: 5, host3: 5, host4: 5),
+ * which could cover the current requested localities. This algorithm will allocate all the
+ * requested containers with no localities.
+ */
+private[yarn] class LocalityPreferredContainerPlacementStrategy(
+    val sparkConf: SparkConf,
+    val yarnConf: Configuration,
+    val resource: Resource) {
+
+  /**
+   * Calculate each container's node locality and rack locality
+   * @param numContainer number of containers to calculate
+   * @param numLocalityAwareTasks number of locality required tasks
+   * @param hostToLocalTaskCount a map to store the preferred hostname and possible task
+   *                             numbers running on it, used as hints for container allocation
+   * @param allocatedHostToContainersMap host to allocated containers map, used to calculate the
+   *                                     expected locality preference by considering the existing
+   *                                     containers
+   * @param localityMatchedPendingAllocations A sequence of pending container request which
+   *                                          matches the localities of current required tasks.
+   * @return node localities and rack localities, each locality is an array of string,
+   *         the length of localities is the same as number of containers
+   */
+  def localityOfRequestedContainers(
+      numContainer: Int,
+      numLocalityAwareTasks: Int,
+      hostToLocalTaskCount: Map[String, Int],
+      allocatedHostToContainersMap: HashMap[String, Set[ContainerId]],
+      localityMatchedPendingAllocations: Seq[ContainerRequest]
+    ): Array[ContainerLocalityPreferences] = {
+    val updatedHostToContainerCount = expectedHostToContainerCount(
+      numLocalityAwareTasks, hostToLocalTaskCount, allocatedHostToContainersMap,
+        localityMatchedPendingAllocations)
+    val updatedLocalityAwareContainerNum = updatedHostToContainerCount.values.sum
+
+    // The number of containers to allocate, divided into two groups, one with preferred locality,
+    // and the other without locality preference.
+    val requiredLocalityFreeContainerNum =
+      math.max(0, numContainer - updatedLocalityAwareContainerNum)
+    val requiredLocalityAwareContainerNum = numContainer - requiredLocalityFreeContainerNum
+
+    val containerLocalityPreferences = ArrayBuffer[ContainerLocalityPreferences]()
+    if (requiredLocalityFreeContainerNum > 0) {
+      for (i <- 0 until requiredLocalityFreeContainerNum) {
+        containerLocalityPreferences += ContainerLocalityPreferences(
+          null.asInstanceOf[Array[String]], null.asInstanceOf[Array[String]])
+      }
+    }
+
+    if (requiredLocalityAwareContainerNum > 0) {
+      val largestRatio = updatedHostToContainerCount.values.max
+      // Round the ratio of preferred locality to the number of locality required container
+      // number, which is used for locality preferred host calculating.
+      var preferredLocalityRatio = updatedHostToContainerCount.mapValues { ratio =>
+        val adjustedRatio = ratio.toDouble * requiredLocalityAwareContainerNum / largestRatio
+        adjustedRatio.ceil.toInt
+      }
+
+      for (i <- 0 until requiredLocalityAwareContainerNum) {
+        // 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 =>
+          RackResolver.resolve(yarnConf, h).getNetworkLocation
+        }.toSet
+        containerLocalityPreferences += ContainerLocalityPreferences(hosts, racks.toArray)
+
+        // Minus 1 each time when the host is used. When the current ratio is 0,
+        // which means all the required ratio is satisfied, this host will not be allocated again.
+        preferredLocalityRatio = preferredLocalityRatio.mapValues(_ - 1)
+      }
+    }
+
+    containerLocalityPreferences.toArray
+  }
+
+  /**
+   * Calculate the number of executors need to satisfy the given number of pending tasks.
+   */
+  private def numExecutorsPending(numTasksPending: Int): Int = {
+    val coresPerExecutor = resource.getVirtualCores
+    (numTasksPending * sparkConf.get(CPUS_PER_TASK) + coresPerExecutor - 1) / coresPerExecutor
+  }
+
+  /**
+   * Calculate the expected host to number of containers by considering with allocated containers.
+   * @param localityAwareTasks number of locality aware tasks
+   * @param hostToLocalTaskCount a map to store the preferred hostname and possible task
+   *                             numbers running on it, used as hints for container allocation
+   * @param allocatedHostToContainersMap host to allocated containers map, used to calculate the
+   *                                     expected locality preference by considering the existing
+   *                                     containers
+   * @param localityMatchedPendingAllocations A sequence of pending container request which
+   *                                          matches the localities of current required tasks.
+   * @return a map with hostname as key and required number of containers on this host as value
+   */
+  private def expectedHostToContainerCount(
+      localityAwareTasks: Int,
+      hostToLocalTaskCount: Map[String, Int],
+      allocatedHostToContainersMap: HashMap[String, Set[ContainerId]],
+      localityMatchedPendingAllocations: Seq[ContainerRequest]
+    ): Map[String, Int] = {
+    val totalLocalTaskNum = hostToLocalTaskCount.values.sum
+    val pendingHostToContainersMap = pendingHostToContainerCount(localityMatchedPendingAllocations)
+
+    hostToLocalTaskCount.map { case (host, count) =>
+      val expectedCount =
+        count.toDouble * numExecutorsPending(localityAwareTasks) / totalLocalTaskNum
+      // Take the locality of pending containers into consideration
+      val existedCount = allocatedHostToContainersMap.get(host).map(_.size).getOrElse(0) +
+        pendingHostToContainersMap.getOrElse(host, 0.0)
+
+      // If existing container can not fully satisfy the expected number of container,
+      // the required container number is expected count minus existed count. Otherwise the
+      // required container number is 0.
+      (host, math.max(0, (expectedCount - existedCount).ceil.toInt))
+    }
+  }
+
+  /**
+   * According to the locality ratio and number of container requests, calculate the host to
+   * possible number of containers for pending allocated containers.
+   *
+   * If current locality ratio of hosts is: Host1 : Host2 : Host3 = 20 : 20 : 10,
+   * and pending container requests is 3, so the possible number of containers on
+   * Host1 : Host2 : Host3 will be 1.2 : 1.2 : 0.6.
+   * @param localityMatchedPendingAllocations A sequence of pending container request which
+   *                                          matches the localities of current required tasks.
+   * @return a Map with hostname as key and possible number of containers on this host as value
+   */
+  private def pendingHostToContainerCount(
+      localityMatchedPendingAllocations: Seq[ContainerRequest]): Map[String, Double] = {
+    val pendingHostToContainerCount = new HashMap[String, Int]()
+    localityMatchedPendingAllocations.foreach { cr =>
+      cr.getNodes.asScala.foreach { n =>
+        val count = pendingHostToContainerCount.getOrElse(n, 0) + 1
+        pendingHostToContainerCount(n) = count
+      }
+    }
+
+    val possibleTotalContainerNum = pendingHostToContainerCount.values.sum
+    val localityMatchedPendingNum = localityMatchedPendingAllocations.size.toDouble
+    pendingHostToContainerCount.mapValues(_ * localityMatchedPendingNum / possibleTotalContainerNum)
+      .toMap
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..0b66d1c
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
@@ -0,0 +1,727 @@
+/*
+ * 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 java.util.Collections
+import java.util.concurrent._
+import java.util.regex.Pattern
+
+import scala.collection.mutable
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue}
+import scala.collection.JavaConverters._
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.AMRMClient
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.RackResolver
+import org.apache.log4j.{Level, Logger}
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkException}
+import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef}
+import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason}
+import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor
+import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RetrieveLastAllocatedExecutorId
+import org.apache.spark.util.{Clock, SystemClock, ThreadUtils}
+
+/**
+ * YarnAllocator is charged with requesting containers from the YARN ResourceManager and deciding
+ * what to do with containers when YARN fulfills these requests.
+ *
+ * This class makes use of YARN's AMRMClient APIs. We interact with the AMRMClient in three ways:
+ * * Making our resource needs known, which updates local bookkeeping about containers requested.
+ * * Calling "allocate", which syncs our local container requests with the RM, and returns any
+ *   containers that YARN has granted to us.  This also functions as a heartbeat.
+ * * Processing the containers granted to us to possibly launch executors inside of them.
+ *
+ * The public methods of this class are thread-safe.  All methods that mutate state are
+ * synchronized.
+ */
+private[yarn] class YarnAllocator(
+    driverUrl: String,
+    driverRef: RpcEndpointRef,
+    conf: YarnConfiguration,
+    sparkConf: SparkConf,
+    amClient: AMRMClient[ContainerRequest],
+    appAttemptId: ApplicationAttemptId,
+    securityMgr: SecurityManager,
+    localResources: Map[String, LocalResource])
+  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]
+
+  // Containers that we no longer care about. We've either already told the RM to release them or
+  // will on the next heartbeat. Containers get removed from this map after the RM tells us they've
+  // completed.
+  private val releasedContainers = Collections.newSetFromMap[ContainerId](
+    new ConcurrentHashMap[ContainerId, java.lang.Boolean])
+
+  @volatile private var numExecutorsRunning = 0
+
+  /**
+   * Used to generate a unique ID per executor
+   *
+   * Init `executorIdCounter`. when AM restart, `executorIdCounter` will reset to 0. Then
+   * the id of new executor will start from 1, this will conflict with the executor has
+   * already created before. So, we should initialize the `executorIdCounter` by getting
+   * the max executorId from driver.
+   *
+   * And this situation of executorId conflict is just in yarn client mode, so this is an issue
+   * in yarn client mode. For more details, can check in jira.
+   *
+   * @see SPARK-12864
+   */
+  private var executorIdCounter: Int =
+    driverRef.askWithRetry[Int](RetrieveLastAllocatedExecutorId)
+
+  // Queue to store the timestamp of failed executors
+  private val failedExecutorsTimeStamps = new Queue[Long]()
+
+  private var clock: Clock = new SystemClock
+
+  private val executorFailuresValidityInterval =
+    sparkConf.get(EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS).getOrElse(-1L)
+
+  @volatile private var targetNumExecutors =
+    YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sparkConf)
+
+  // Executor loss reason requests that are pending - maps from executor ID for inquiry to a
+  // list of requesters that should be responded to once we find out why the given executor
+  // was lost.
+  private val pendingLossReasonRequests = new HashMap[String, mutable.Buffer[RpcCallContext]]
+
+  // Maintain loss reasons for already released executors, it will be added when executor loss
+  // reason is got from AM-RM call, and be removed after querying this loss reason.
+  private val releasedExecutorLossReasons = new HashMap[String, ExecutorLossReason]
+
+  // Keep track of which container is running which executor to remove the executors later
+  // Visible for testing.
+  private[yarn] val executorIdToContainer = new HashMap[String, Container]
+
+  private var numUnexpectedContainerRelease = 0L
+  private val containerIdToExecutorId = new HashMap[ContainerId, String]
+
+  // Executor memory in MB.
+  protected val executorMemory = sparkConf.get(EXECUTOR_MEMORY).toInt
+  // Additional memory overhead.
+  protected val memoryOverhead: Int = sparkConf.get(EXECUTOR_MEMORY_OVERHEAD).getOrElse(
+    math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toInt, MEMORY_OVERHEAD_MIN)).toInt
+  // Number of cores per executor.
+  protected val executorCores = sparkConf.get(EXECUTOR_CORES)
+  // Resource capability requested for each executors
+  private[yarn] val resource = Resource.newInstance(executorMemory + memoryOverhead, executorCores)
+
+  private val launcherPool = ThreadUtils.newDaemonCachedThreadPool(
+    "ContainerLauncher", sparkConf.get(CONTAINER_LAUNCH_MAX_THREADS))
+
+  // For testing
+  private val launchContainers = sparkConf.getBoolean("spark.yarn.launchContainers", true)
+
+  private val labelExpression = sparkConf.get(EXECUTOR_NODE_LABEL_EXPRESSION)
+
+  // ContainerRequest constructor that can take a node label expression. We grab it through
+  // reflection because it's only available in later versions of YARN.
+  private val nodeLabelConstructor = labelExpression.flatMap { expr =>
+    try {
+      Some(classOf[ContainerRequest].getConstructor(classOf[Resource],
+        classOf[Array[String]], classOf[Array[String]], classOf[Priority], classOf[Boolean],
+        classOf[String]))
+    } catch {
+      case e: NoSuchMethodException =>
+        logWarning(s"Node label expression $expr will be ignored because YARN version on" +
+          " classpath does not support it.")
+        None
+    }
+  }
+
+  // A map to store preferred hostname and possible task numbers running on it.
+  private var hostToLocalTaskCounts: Map[String, Int] = Map.empty
+
+  // Number of tasks that have locality preferences in active stages
+  private var numLocalityAwareTasks: Int = 0
+
+  // A container placement strategy based on pending tasks' locality preference
+  private[yarn] val containerPlacementStrategy =
+    new LocalityPreferredContainerPlacementStrategy(sparkConf, conf, resource)
+
+  /**
+   * Use a different clock for YarnAllocator. This is mainly used for testing.
+   */
+  def setClock(newClock: Clock): Unit = {
+    clock = newClock
+  }
+
+  def getNumExecutorsRunning: Int = numExecutorsRunning
+
+  def getNumExecutorsFailed: Int = synchronized {
+    val endTime = clock.getTimeMillis()
+
+    while (executorFailuresValidityInterval > 0
+      && failedExecutorsTimeStamps.nonEmpty
+      && failedExecutorsTimeStamps.head < endTime - executorFailuresValidityInterval) {
+      failedExecutorsTimeStamps.dequeue()
+    }
+
+    failedExecutorsTimeStamps.size
+  }
+
+  /**
+   * A sequence of pending container requests that have not yet been fulfilled.
+   */
+  def getPendingAllocate: Seq[ContainerRequest] = getPendingAtLocation(ANY_HOST)
+
+  /**
+   * A sequence of pending container requests at the given location that have not yet been
+   * fulfilled.
+   */
+  private def getPendingAtLocation(location: String): Seq[ContainerRequest] = {
+    amClient.getMatchingRequests(RM_REQUEST_PRIORITY, location, resource).asScala
+      .flatMap(_.asScala)
+      .toSeq
+  }
+
+  /**
+   * Request as many executors from the ResourceManager as needed to reach the desired total. If
+   * the requested total is smaller than the current number of running executors, no executors will
+   * be killed.
+   * @param requestedTotal total number of containers requested
+   * @param localityAwareTasks number of locality aware tasks to be used as container placement hint
+   * @param hostToLocalTaskCount a map of preferred hostname to possible task counts to be used as
+   *                             container placement hint.
+   * @return Whether the new requested total is different than the old value.
+   */
+  def requestTotalExecutorsWithPreferredLocalities(
+      requestedTotal: Int,
+      localityAwareTasks: Int,
+      hostToLocalTaskCount: Map[String, Int]): Boolean = synchronized {
+    this.numLocalityAwareTasks = localityAwareTasks
+    this.hostToLocalTaskCounts = hostToLocalTaskCount
+
+    if (requestedTotal != targetNumExecutors) {
+      logInfo(s"Driver requested a total number of $requestedTotal executor(s).")
+      targetNumExecutors = requestedTotal
+      true
+    } else {
+      false
+    }
+  }
+
+  /**
+   * Request that the ResourceManager release the container running the specified executor.
+   */
+  def killExecutor(executorId: String): Unit = synchronized {
+    if (executorIdToContainer.contains(executorId)) {
+      val container = executorIdToContainer.get(executorId).get
+      internalReleaseContainer(container)
+      numExecutorsRunning -= 1
+    } else {
+      logWarning(s"Attempted to kill unknown executor $executorId!")
+    }
+  }
+
+  /**
+   * Request resources such that, if YARN gives us all we ask for, we'll have a number of containers
+   * equal to maxExecutors.
+   *
+   * Deal with any containers YARN has granted to us by possibly launching executors in them.
+   *
+   * This must be synchronized because variables read in this method are mutated by other methods.
+   */
+  def allocateResources(): Unit = synchronized {
+    updateResourceRequests()
+
+    val progressIndicator = 0.1f
+    // Poll the ResourceManager. This doubles as a heartbeat if there are no pending container
+    // requests.
+    val allocateResponse = amClient.allocate(progressIndicator)
+
+    val allocatedContainers = allocateResponse.getAllocatedContainers()
+
+    if (allocatedContainers.size > 0) {
+      logDebug("Allocated containers: %d. Current executor count: %d. Cluster resources: %s."
+        .format(
+          allocatedContainers.size,
+          numExecutorsRunning,
+          allocateResponse.getAvailableResources))
+
+      handleAllocatedContainers(allocatedContainers.asScala)
+    }
+
+    val completedContainers = allocateResponse.getCompletedContainersStatuses()
+    if (completedContainers.size > 0) {
+      logDebug("Completed %d containers".format(completedContainers.size))
+      processCompletedContainers(completedContainers.asScala)
+      logDebug("Finished processing %d completed containers. Current running executor count: %d."
+        .format(completedContainers.size, numExecutorsRunning))
+    }
+  }
+
+  /**
+   * Update the set of container requests that we will sync with the RM based on the number of
+   * executors we have currently running and our target number of executors.
+   *
+   * Visible for testing.
+   */
+  def updateResourceRequests(): Unit = {
+    val pendingAllocate = getPendingAllocate
+    val numPendingAllocate = pendingAllocate.size
+    val missing = targetNumExecutors - numPendingAllocate - numExecutorsRunning
+
+    if (missing > 0) {
+      logInfo(s"Will request $missing executor container(s), each with " +
+        s"${resource.getVirtualCores} core(s) and " +
+        s"${resource.getMemory} MB memory (including $memoryOverhead MB of overhead)")
+
+      // Split the pending container request into three groups: locality matched list, locality
+      // unmatched list and non-locality list. Take the locality matched container request into
+      // consideration of container placement, treat as allocated containers.
+      // For locality unmatched and locality free container requests, cancel these container
+      // requests, since required locality preference has been changed, recalculating using
+      // container placement strategy.
+      val (localRequests, staleRequests, anyHostRequests) = splitPendingAllocationsByLocality(
+        hostToLocalTaskCounts, pendingAllocate)
+
+      // cancel "stale" requests for locations that are no longer needed
+      staleRequests.foreach { stale =>
+        amClient.removeContainerRequest(stale)
+      }
+      val cancelledContainers = staleRequests.size
+      if (cancelledContainers > 0) {
+        logInfo(s"Canceled $cancelledContainers container request(s) (locality no longer needed)")
+      }
+
+      // consider the number of new containers and cancelled stale containers available
+      val availableContainers = missing + cancelledContainers
+
+      // to maximize locality, include requests with no locality preference that can be cancelled
+      val potentialContainers = availableContainers + anyHostRequests.size
+
+      val containerLocalityPreferences = containerPlacementStrategy.localityOfRequestedContainers(
+        potentialContainers, numLocalityAwareTasks, hostToLocalTaskCounts,
+          allocatedHostToContainersMap, localRequests)
+
+      val newLocalityRequests = new mutable.ArrayBuffer[ContainerRequest]
+      containerLocalityPreferences.foreach {
+        case ContainerLocalityPreferences(nodes, racks) if nodes != null =>
+          newLocalityRequests += createContainerRequest(resource, nodes, racks)
+        case _ =>
+      }
+
+      if (availableContainers >= newLocalityRequests.size) {
+        // more containers are available than needed for locality, fill in requests for any host
+        for (i <- 0 until (availableContainers - newLocalityRequests.size)) {
+          newLocalityRequests += createContainerRequest(resource, null, null)
+        }
+      } else {
+        val numToCancel = newLocalityRequests.size - availableContainers
+        // cancel some requests without locality preferences to schedule more local containers
+        anyHostRequests.slice(0, numToCancel).foreach { nonLocal =>
+          amClient.removeContainerRequest(nonLocal)
+        }
+        if (numToCancel > 0) {
+          logInfo(s"Canceled $numToCancel unlocalized container requests to resubmit with locality")
+        }
+      }
+
+      newLocalityRequests.foreach { request =>
+        amClient.addContainerRequest(request)
+      }
+
+      if (log.isInfoEnabled()) {
+        val (localized, anyHost) = newLocalityRequests.partition(_.getNodes() != null)
+        if (anyHost.nonEmpty) {
+          logInfo(s"Submitted ${anyHost.size} unlocalized container requests.")
+        }
+        localized.foreach { request =>
+          logInfo(s"Submitted container request for host ${hostStr(request)}.")
+        }
+      }
+    } else if (numPendingAllocate > 0 && missing < 0) {
+      val numToCancel = math.min(numPendingAllocate, -missing)
+      logInfo(s"Canceling requests for $numToCancel executor container(s) to have a new desired " +
+        s"total $targetNumExecutors executors.")
+
+      val matchingRequests = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, ANY_HOST, resource)
+      if (!matchingRequests.isEmpty) {
+        matchingRequests.iterator().next().asScala
+          .take(numToCancel).foreach(amClient.removeContainerRequest)
+      } else {
+        logWarning("Expected to find pending requests, but found none.")
+      }
+    }
+  }
+
+  private def hostStr(request: ContainerRequest): String = {
+    Option(request.getNodes) match {
+      case Some(nodes) => nodes.asScala.mkString(",")
+      case None => "Any"
+    }
+  }
+
+  /**
+   * Creates a container request, handling the reflection required to use YARN features that were
+   * added in recent versions.
+   */
+  private def createContainerRequest(
+      resource: Resource,
+      nodes: Array[String],
+      racks: Array[String]): ContainerRequest = {
+    nodeLabelConstructor.map { constructor =>
+      constructor.newInstance(resource, nodes, racks, RM_REQUEST_PRIORITY, true: java.lang.Boolean,
+        labelExpression.orNull)
+    }.getOrElse(new ContainerRequest(resource, nodes, racks, RM_REQUEST_PRIORITY))
+  }
+
+  /**
+   * Handle containers granted by the RM by launching executors on them.
+   *
+   * Due to the way the YARN allocation protocol works, certain healthy race conditions can result
+   * in YARN granting containers that we no longer need. In this case, we release them.
+   *
+   * Visible for testing.
+   */
+  def handleAllocatedContainers(allocatedContainers: Seq[Container]): Unit = {
+    val containersToUse = new ArrayBuffer[Container](allocatedContainers.size)
+
+    // Match incoming requests by host
+    val remainingAfterHostMatches = new ArrayBuffer[Container]
+    for (allocatedContainer <- allocatedContainers) {
+      matchContainerToRequest(allocatedContainer, allocatedContainer.getNodeId.getHost,
+        containersToUse, remainingAfterHostMatches)
+    }
+
+    // Match remaining by rack
+    val remainingAfterRackMatches = new ArrayBuffer[Container]
+    for (allocatedContainer <- remainingAfterHostMatches) {
+      val rack = RackResolver.resolve(conf, allocatedContainer.getNodeId.getHost).getNetworkLocation
+      matchContainerToRequest(allocatedContainer, rack, containersToUse,
+        remainingAfterRackMatches)
+    }
+
+    // Assign remaining that are neither node-local nor rack-local
+    val remainingAfterOffRackMatches = new ArrayBuffer[Container]
+    for (allocatedContainer <- remainingAfterRackMatches) {
+      matchContainerToRequest(allocatedContainer, ANY_HOST, containersToUse,
+        remainingAfterOffRackMatches)
+    }
+
+    if (!remainingAfterOffRackMatches.isEmpty) {
+      logDebug(s"Releasing ${remainingAfterOffRackMatches.size} unneeded containers that were " +
+        s"allocated to us")
+      for (container <- remainingAfterOffRackMatches) {
+        internalReleaseContainer(container)
+      }
+    }
+
+    runAllocatedContainers(containersToUse)
+
+    logInfo("Received %d containers from YARN, launching executors on %d of them."
+      .format(allocatedContainers.size, containersToUse.size))
+  }
+
+  /**
+   * Looks for requests for the given location that match the given container allocation. If it
+   * finds one, removes the request so that it won't be submitted again. Places the container into
+   * containersToUse or remaining.
+   *
+   * @param allocatedContainer container that was given to us by YARN
+   * @param location resource name, either a node, rack, or *
+   * @param containersToUse list of containers that will be used
+   * @param remaining list of containers that will not be used
+   */
+  private def matchContainerToRequest(
+      allocatedContainer: Container,
+      location: String,
+      containersToUse: ArrayBuffer[Container],
+      remaining: ArrayBuffer[Container]): Unit = {
+    // SPARK-6050: certain Yarn configurations return a virtual core count that doesn't match the
+    // request; for example, capacity scheduler + DefaultResourceCalculator. So match on requested
+    // memory, but use the asked vcore count for matching, effectively disabling matching on vcore
+    // count.
+    val matchingResource = Resource.newInstance(allocatedContainer.getResource.getMemory,
+          resource.getVirtualCores)
+    val matchingRequests = amClient.getMatchingRequests(allocatedContainer.getPriority, location,
+      matchingResource)
+
+    // Match the allocation to a request
+    if (!matchingRequests.isEmpty) {
+      val containerRequest = matchingRequests.get(0).iterator.next
+      amClient.removeContainerRequest(containerRequest)
+      containersToUse += allocatedContainer
+    } else {
+      remaining += allocatedContainer
+    }
+  }
+
+  /**
+   * Launches executors in the allocated containers.
+   */
+  private def runAllocatedContainers(containersToUse: ArrayBuffer[Container]): Unit = {
+    for (container <- containersToUse) {
+      executorIdCounter += 1
+      val executorHostname = container.getNodeId.getHost
+      val containerId = container.getId
+      val executorId = executorIdCounter.toString
+      assert(container.getResource.getMemory >= resource.getMemory)
+      logInfo(s"Launching container $containerId on host $executorHostname")
+
+      def updateInternalState(): Unit = synchronized {
+        numExecutorsRunning += 1
+        executorIdToContainer(executorId) = container
+        containerIdToExecutorId(container.getId) = executorId
+
+        val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname,
+          new HashSet[ContainerId])
+        containerSet += containerId
+        allocatedContainerToHostMap.put(containerId, executorHostname)
+      }
+
+      if (numExecutorsRunning < targetNumExecutors) {
+        if (launchContainers) {
+          launcherPool.execute(new Runnable {
+            override def run(): Unit = {
+              try {
+                new ExecutorRunnable(
+                  Some(container),
+                  conf,
+                  sparkConf,
+                  driverUrl,
+                  executorId,
+                  executorHostname,
+                  executorMemory,
+                  executorCores,
+                  appAttemptId.getApplicationId.toString,
+                  securityMgr,
+                  localResources
+                ).run()
+                updateInternalState()
+              } catch {
+                case NonFatal(e) =>
+                  logError(s"Failed to launch executor $executorId on container $containerId", e)
+                  // Assigned container should be released immediately to avoid unnecessary resource
+                  // occupation.
+                  amClient.releaseAssignedContainer(containerId)
+              }
+            }
+          })
+        } else {
+          // For test only
+          updateInternalState()
+        }
+      } else {
+        logInfo(("Skip launching executorRunnable as runnning Excecutors count: %d " +
+          "reached target Executors count: %d.").format(numExecutorsRunning, targetNumExecutors))
+      }
+    }
+  }
+
+  // Visible for testing.
+  private[yarn] def processCompletedContainers(completedContainers: Seq[ContainerStatus]): Unit = {
+    for (completedContainer <- completedContainers) {
+      val containerId = completedContainer.getContainerId
+      val alreadyReleased = releasedContainers.remove(containerId)
+      val hostOpt = allocatedContainerToHostMap.get(containerId)
+      val onHostStr = hostOpt.map(host => s" on host: $host").getOrElse("")
+      val exitReason = if (!alreadyReleased) {
+        // Decrement the number of executors running. The next iteration of
+        // the ApplicationMaster's reporting thread will take care of allocating.
+        numExecutorsRunning -= 1
+        logInfo("Completed container %s%s (state: %s, exit status: %s)".format(
+          containerId,
+          onHostStr,
+          completedContainer.getState,
+          completedContainer.getExitStatus))
+        // Hadoop 2.2.X added a ContainerExitStatus we should switch to use
+        // there are some exit status' we shouldn't necessarily count against us, but for
+        // now I think its ok as none of the containers are expected to exit.
+        val exitStatus = completedContainer.getExitStatus
+        val (exitCausedByApp, containerExitReason) = exitStatus match {
+          case ContainerExitStatus.SUCCESS =>
+            (false, s"Executor for container $containerId exited because of a YARN event (e.g., " +
+              "pre-emption) and not because of an error in the running job.")
+          case ContainerExitStatus.PREEMPTED =>
+            // Preemption is not the fault of the running tasks, since YARN preempts containers
+            // merely to do resource sharing, and tasks that fail due to preempted executors could
+            // just as easily finish on any other executor. See SPARK-8167.
+            (false, s"Container ${containerId}${onHostStr} was preempted.")
+          // Should probably still count memory exceeded exit codes towards task failures
+          case VMEM_EXCEEDED_EXIT_CODE =>
+            (true, memLimitExceededLogMessage(
+              completedContainer.getDiagnostics,
+              VMEM_EXCEEDED_PATTERN))
+          case PMEM_EXCEEDED_EXIT_CODE =>
+            (true, memLimitExceededLogMessage(
+              completedContainer.getDiagnostics,
+              PMEM_EXCEEDED_PATTERN))
+          case _ =>
+            // Enqueue the timestamp of failed executor
+            failedExecutorsTimeStamps.enqueue(clock.getTimeMillis())
+            (true, "Container marked as failed: " + containerId + onHostStr +
+              ". Exit status: " + completedContainer.getExitStatus +
+              ". Diagnostics: " + completedContainer.getDiagnostics)
+
+        }
+        if (exitCausedByApp) {
+          logWarning(containerExitReason)
+        } else {
+          logInfo(containerExitReason)
+        }
+        ExecutorExited(exitStatus, exitCausedByApp, containerExitReason)
+      } else {
+        // If we have already released this container, then it must mean
+        // that the driver has explicitly requested it to be killed
+        ExecutorExited(completedContainer.getExitStatus, exitCausedByApp = false,
+          s"Container $containerId exited from explicit termination request.")
+      }
+
+      for {
+        host <- hostOpt
+        containerSet <- allocatedHostToContainersMap.get(host)
+      } {
+        containerSet.remove(containerId)
+        if (containerSet.isEmpty) {
+          allocatedHostToContainersMap.remove(host)
+        } else {
+          allocatedHostToContainersMap.update(host, containerSet)
+        }
+
+        allocatedContainerToHostMap.remove(containerId)
+      }
+
+      containerIdToExecutorId.remove(containerId).foreach { eid =>
+        executorIdToContainer.remove(eid)
+        pendingLossReasonRequests.remove(eid) match {
+          case Some(pendingRequests) =>
+            // Notify application of executor loss reasons so it can decide whether it should abort
+            pendingRequests.foreach(_.reply(exitReason))
+
+          case None =>
+            // We cannot find executor for pending reasons. This is because completed container
+            // is processed before querying pending result. We should store it for later query.
+            // This is usually happened when explicitly killing a container, the result will be
+            // returned in one AM-RM communication. So query RPC will be later than this completed
+            // container process.
+            releasedExecutorLossReasons.put(eid, exitReason)
+        }
+        if (!alreadyReleased) {
+          // The executor could have gone away (like no route to host, node failure, etc)
+          // Notify backend about the failure of the executor
+          numUnexpectedContainerRelease += 1
+          driverRef.send(RemoveExecutor(eid, exitReason))
+        }
+      }
+    }
+  }
+
+  /**
+   * Register that some RpcCallContext has asked the AM why the executor was lost. Note that
+   * we can only find the loss reason to send back in the next call to allocateResources().
+   */
+  private[yarn] def enqueueGetLossReasonRequest(
+      eid: String,
+      context: RpcCallContext): Unit = synchronized {
+    if (executorIdToContainer.contains(eid)) {
+      pendingLossReasonRequests
+        .getOrElseUpdate(eid, new ArrayBuffer[RpcCallContext]) += context
+    } else if (releasedExecutorLossReasons.contains(eid)) {
+      // Executor is already released explicitly before getting the loss reason, so directly send
+      // the pre-stored lost reason
+      context.reply(releasedExecutorLossReasons.remove(eid).get)
+    } else {
+      logWarning(s"Tried to get the loss reason for non-existent executor $eid")
+      context.sendFailure(
+        new SparkException(s"Fail to find loss reason for non-existent executor $eid"))
+    }
+  }
+
+  private def internalReleaseContainer(container: Container): Unit = {
+    releasedContainers.add(container.getId())
+    amClient.releaseAssignedContainer(container.getId())
+  }
+
+  private[yarn] def getNumUnexpectedContainerRelease = numUnexpectedContainerRelease
+
+  private[yarn] def getNumPendingLossReasonRequests: Int = synchronized {
+    pendingLossReasonRequests.size
+  }
+
+  /**
+   * Split the pending container requests into 3 groups based on current localities of pending
+   * tasks.
+   * @param hostToLocalTaskCount a map of preferred hostname to possible task counts to be used as
+   *                             container placement hint.
+   * @param pendingAllocations A sequence of pending allocation container request.
+   * @return A tuple of 3 sequences, first is a sequence of locality matched container
+   *         requests, second is a sequence of locality unmatched container requests, and third is a
+   *         sequence of locality free container requests.
+   */
+  private def splitPendingAllocationsByLocality(
+      hostToLocalTaskCount: Map[String, Int],
+      pendingAllocations: Seq[ContainerRequest]
+    ): (Seq[ContainerRequest], Seq[ContainerRequest], Seq[ContainerRequest]) = {
+    val localityMatched = ArrayBuffer[ContainerRequest]()
+    val localityUnMatched = ArrayBuffer[ContainerRequest]()
+    val localityFree = ArrayBuffer[ContainerRequest]()
+
+    val preferredHosts = hostToLocalTaskCount.keySet
+    pendingAllocations.foreach { cr =>
+      val nodes = cr.getNodes
+      if (nodes == null) {
+        localityFree += cr
+      } else if (nodes.asScala.toSet.intersect(preferredHosts).nonEmpty) {
+        localityMatched += cr
+      } else {
+        localityUnMatched += cr
+      }
+    }
+
+    (localityMatched.toSeq, localityUnMatched.toSeq, localityFree.toSeq)
+  }
+
+}
+
+private object YarnAllocator {
+  val MEM_REGEX = "[0-9.]+ [KMG]B"
+  val PMEM_EXCEEDED_PATTERN =
+    Pattern.compile(s"$MEM_REGEX of $MEM_REGEX physical memory used")
+  val VMEM_EXCEEDED_PATTERN =
+    Pattern.compile(s"$MEM_REGEX of $MEM_REGEX virtual memory used")
+  val VMEM_EXCEEDED_EXIT_CODE = -103
+  val PMEM_EXCEEDED_EXIT_CODE = -104
+
+  def memLimitExceededLogMessage(diagnostics: String, pattern: Pattern): String = {
+    val matcher = pattern.matcher(diagnostics)
+    val diag = if (matcher.find()) " " + matcher.group() + "." else ""
+    ("Container killed by YARN for exceeding memory limits." + diag
+      + " Consider boosting spark.yarn.executor.memoryOverhead.")
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..53df11e
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
@@ -0,0 +1,135 @@
+/*
+ * 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 java.util.{List => JList}
+
+import scala.collection.JavaConverters._
+import scala.util.Try
+
+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
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils
+
+import org.apache.spark.{SecurityManager, SparkConf}
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.rpc.RpcEndpointRef
+import org.apache.spark.util.Utils
+
+/**
+ * Handles registering and unregistering the application with the YARN ResourceManager.
+ */
+private[spark] class YarnRMClient extends Logging {
+
+  private var amClient: AMRMClient[ContainerRequest] = _
+  private var uiHistoryAddress: String = _
+  private var registered: Boolean = false
+
+  /**
+   * Registers the application master with the RM.
+   *
+   * @param conf The Yarn configuration.
+   * @param sparkConf The Spark configuration.
+   * @param uiAddress Address of the SparkUI.
+   * @param uiHistoryAddress Address of the application on the History Server.
+   * @param securityMgr The security manager.
+   * @param localResources Map with information about files distributed via YARN's cache.
+   */
+  def register(
+      driverUrl: String,
+      driverRef: RpcEndpointRef,
+      conf: YarnConfiguration,
+      sparkConf: SparkConf,
+      uiAddress: String,
+      uiHistoryAddress: String,
+      securityMgr: SecurityManager,
+      localResources: Map[String, LocalResource]
+    ): YarnAllocator = {
+    amClient = AMRMClient.createAMRMClient()
+    amClient.init(conf)
+    amClient.start()
+    this.uiHistoryAddress = uiHistoryAddress
+
+    logInfo("Registering the ApplicationMaster")
+    synchronized {
+      amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress)
+      registered = true
+    }
+    new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, getAttemptId(), securityMgr,
+      localResources)
+  }
+
+  /**
+   * Unregister the AM. Guaranteed to only be called once.
+   *
+   * @param status The final status of the AM.
+   * @param diagnostics Diagnostics message to include in the final status.
+   */
+  def unregister(status: FinalApplicationStatus, diagnostics: String = ""): Unit = synchronized {
+    if (registered) {
+      amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress)
+    }
+  }
+
+  /** Returns the attempt ID. */
+  def getAttemptId(): ApplicationAttemptId = {
+    YarnSparkHadoopUtil.get.getContainerId.getApplicationAttemptId()
+  }
+
+  /** Returns the configuration for the AmIpFilter to add to the Spark UI. */
+  def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] = {
+    // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2,
+    // so not all stable releases have it.
+    val prefix = Try(classOf[WebAppUtils].getMethod("getHttpSchemePrefix", classOf[Configuration])
+      .invoke(null, conf).asInstanceOf[String]).getOrElse("http://")
+
+    // If running a new enough Yarn, use the HA-aware API for retrieving the RM addresses.
+    try {
+      val method = classOf[WebAppUtils].getMethod("getProxyHostsAndPortsForAmFilter",
+        classOf[Configuration])
+      val proxies = method.invoke(null, conf).asInstanceOf[JList[String]]
+      val hosts = proxies.asScala.map { proxy => proxy.split(":")(0) }
+      val uriBases = proxies.asScala.map { proxy => prefix + proxy + proxyBase }
+      Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(","))
+    } catch {
+      case e: NoSuchMethodException =>
+        val proxy = WebAppUtils.getProxyHostAndPort(conf)
+        val parts = proxy.split(":")
+        val uriBase = prefix + proxy + proxyBase
+        Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase)
+    }
+  }
+
+  /** Returns the maximum number of attempts to register the AM. */
+  def getMaxRegAttempts(sparkConf: SparkConf, yarnConf: YarnConfiguration): Int = {
+    val sparkMaxAttempts = sparkConf.get(MAX_APP_ATTEMPTS).map(_.toInt)
+    val yarnMaxAttempts = yarnConf.getInt(
+      YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)
+    val retval: Int = sparkMaxAttempts match {
+      case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts
+      case None => yarnMaxAttempts
+    }
+
+    retval
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
new file mode 100644
index 0000000..cc53b1b
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
@@ -0,0 +1,317 @@
+/*
+ * 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 java.io.File
+import java.nio.charset.StandardCharsets.UTF_8
+import java.util.regex.Matcher
+import java.util.regex.Pattern
+
+import scala.collection.mutable.{HashMap, ListBuffer}
+import scala.util.Try
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hadoop.security.Credentials
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api.ApplicationConstants
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.records.{ApplicationAccessType, ContainerId, Priority}
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.ConverterUtils
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkException}
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.yarn.security.{ConfigurableCredentialManager, CredentialUpdater}
+import org.apache.spark.internal.config._
+import org.apache.spark.launcher.YarnCommandBuilderUtils
+import org.apache.spark.util.Utils
+
+/**
+ * Contains util methods to interact with Hadoop from spark.
+ */
+class YarnSparkHadoopUtil extends SparkHadoopUtil {
+
+  private var credentialUpdater: CredentialUpdater = _
+
+  override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) {
+    dest.addCredentials(source.getCredentials())
+  }
+
+  // Note that all params which start with SPARK are propagated all the way through, so if in yarn
+  // mode, this MUST be set to true.
+  override def isYarnMode(): Boolean = { true }
+
+  // Return an appropriate (subclass) of Configuration. Creating a config initializes some Hadoop
+  // subsystems. Always create a new config, don't reuse yarnConf.
+  override def newConfiguration(conf: SparkConf): Configuration =
+    new YarnConfiguration(super.newConfiguration(conf))
+
+  // Add any user credentials to the job conf which are necessary for running on a secure Hadoop
+  // cluster
+  override def addCredentials(conf: JobConf) {
+    val jobCreds = conf.getCredentials()
+    jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials())
+  }
+
+  override def getCurrentUserCredentials(): Credentials = {
+    UserGroupInformation.getCurrentUser().getCredentials()
+  }
+
+  override def addCurrentUserCredentials(creds: Credentials) {
+    UserGroupInformation.getCurrentUser().addCredentials(creds)
+  }
+
+  override def addSecretKeyToUserCredentials(key: String, secret: String) {
+    val creds = new Credentials()
+    creds.addSecretKey(new Text(key), secret.getBytes(UTF_8))
+    addCurrentUserCredentials(creds)
+  }
+
+  override def getSecretKeyFromUserCredentials(key: String): Array[Byte] = {
+    val credentials = getCurrentUserCredentials()
+    if (credentials != null) credentials.getSecretKey(new Text(key)) else null
+  }
+
+  private[spark] override def startCredentialUpdater(sparkConf: SparkConf): Unit = {
+    credentialUpdater =
+      new ConfigurableCredentialManager(sparkConf, newConfiguration(sparkConf)).credentialUpdater()
+    credentialUpdater.start()
+  }
+
+  private[spark] override def stopCredentialUpdater(): Unit = {
+    if (credentialUpdater != null) {
+      credentialUpdater.stop()
+      credentialUpdater = null
+    }
+  }
+
+  private[spark] def getContainerId: ContainerId = {
+    val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name())
+    ConverterUtils.toContainerId(containerIdString)
+  }
+}
+
+object YarnSparkHadoopUtil {
+  // Additional memory overhead
+  // 10% was arrived at experimentally. In the interest of minimizing memory waste while covering
+  // the common cases. Memory overhead tends to grow with container size.
+
+  val MEMORY_OVERHEAD_FACTOR = 0.10
+  val MEMORY_OVERHEAD_MIN = 384L
+
+  val ANY_HOST = "*"
+
+  val DEFAULT_NUMBER_EXECUTORS = 2
+
+  // All RM requests are issued with same priority : we do not (yet) have any distinction between
+  // request types (like map/reduce in hadoop for example)
+  val RM_REQUEST_PRIORITY = Priority.newInstance(1)
+
+  def get: YarnSparkHadoopUtil = {
+    val yarnMode = java.lang.Boolean.parseBoolean(
+      System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
+    if (!yarnMode) {
+      throw new SparkException("YarnSparkHadoopUtil is not available in non-YARN mode!")
+    }
+    SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil]
+  }
+  /**
+   * Add a path variable to the given environment map.
+   * If the map already contains this key, append the value to the existing value instead.
+   */
+  def addPathToEnvironment(env: HashMap[String, String], key: String, value: String): Unit = {
+    val newValue = if (env.contains(key)) { env(key) + getClassPathSeparator  + value } else value
+    env.put(key, newValue)
+  }
+
+  /**
+   * Set zero or more environment variables specified by the given input string.
+   * The input string is expected to take the form "KEY1=VAL1,KEY2=VAL2,KEY3=VAL3".
+   */
+  def setEnvFromInputString(env: HashMap[String, String], inputString: String): Unit = {
+    if (inputString != null && inputString.length() > 0) {
+      val childEnvs = inputString.split(",")
+      val p = Pattern.compile(environmentVariableRegex)
+      for (cEnv <- childEnvs) {
+        val parts = cEnv.split("=") // split on '='
+        val m = p.matcher(parts(1))
+        val sb = new StringBuffer
+        while (m.find()) {
+          val variable = m.group(1)
+          var replace = ""
+          if (env.get(variable) != None) {
+            replace = env.get(variable).get
+          } else {
+            // if this key is not configured for the child .. get it from the env
+            replace = System.getenv(variable)
+            if (replace == null) {
+            // the env key is note present anywhere .. simply set it
+              replace = ""
+            }
+          }
+          m.appendReplacement(sb, Matcher.quoteReplacement(replace))
+        }
+        m.appendTail(sb)
+        // This treats the environment variable as path variable delimited by `File.pathSeparator`
+        // This is kept for backward compatibility and consistency with Hadoop's behavior
+        addPathToEnvironment(env, parts(0), sb.toString)
+      }
+    }
+  }
+
+  private val environmentVariableRegex: String = {
+    if (Utils.isWindows) {
+      "%([A-Za-z_][A-Za-z0-9_]*?)%"
+    } else {
+      "\\$([A-Za-z_][A-Za-z0-9_]*)"
+    }
+  }
+
+  /**
+   * Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
+   * Not killing the task leaves various aspects of the executor and (to some extent) the jvm in
+   * an inconsistent state.
+   * TODO: If the OOM is not recoverable by rescheduling it on different node, then do
+   * 'something' to fail job ... akin to blacklisting trackers in mapred ?
+   *
+   * The handler if an OOM Exception is thrown by the JVM must be configured on Windows
+   * differently: the 'taskkill' command should be used, whereas Unix-based systems use 'kill'.
+   *
+   * As the JVM interprets both %p and %%p as the same, we can use either of them. However,
+   * some tests on Windows computers suggest, that the JVM only accepts '%%p'.
+   *
+   * Furthermore, the behavior of the character '%' on the Windows command line differs from
+   * the behavior of '%' in a .cmd file: it gets interpreted as an incomplete environment
+   * variable. Windows .cmd files escape a '%' by '%%'. Thus, the correct way of writing
+   * '%%p' in an escaped way is '%%%%p'.
+   */
+  private[yarn] def addOutOfMemoryErrorArgument(javaOpts: ListBuffer[String]): Unit = {
+    if (!javaOpts.exists(_.contains("-XX:OnOutOfMemoryError"))) {
+      if (Utils.isWindows) {
+        javaOpts += escapeForShell("-XX:OnOutOfMemoryError=taskkill /F /PID %%%%p")
+      } else {
+        javaOpts += "-XX:OnOutOfMemoryError='kill %p'"
+      }
+    }
+  }
+
+  /**
+   * Escapes a string for inclusion in a command line executed by Yarn. Yarn executes commands
+   * using either
+   *
+   * (Unix-based) `bash -c "command arg1 arg2"` and that means plain quoting doesn't really work.
+   * The argument is enclosed in single quotes and some key characters are escaped.
+   *
+   * (Windows-based) part of a .cmd file in which case windows escaping for each argument must be
+   * applied. Windows is quite lenient, however it is usually Java that causes trouble, needing to
+   * distinguish between arguments starting with '-' and class names. If arguments are surrounded
+   * by ' java takes the following string as is, hence an argument is mistakenly taken as a class
+   * name which happens to start with a '-'. The way to avoid this, is to surround nothing with
+   * a ', but instead with a ".
+   *
+   * @param arg A single argument.
+   * @return Argument quoted for execution via Yarn's generated shell script.
+   */
+  def escapeForShell(arg: String): String = {
+    if (arg != null) {
+      if (Utils.isWindows) {
+        YarnCommandBuilderUtils.quoteForBatchScript(arg)
+      } else {
+        val escaped = new StringBuilder("'")
+        for (i <- 0 to arg.length() - 1) {
+          arg.charAt(i) match {
+            case '$' => escaped.append("\\$")
+            case '"' => escaped.append("\\\"")
+            case '\'' => escaped.append("'\\''")
+            case c => escaped.append(c)
+          }
+        }
+        escaped.append("'").toString()
+      }
+    } else {
+      arg
+    }
+  }
+
+  // YARN/Hadoop acls are specified as user1,user2 group1,group2
+  // Users and groups are separated by a space and hence we need to pass the acls in same format
+  def getApplicationAclsForYarn(securityMgr: SecurityManager)
+      : Map[ApplicationAccessType, String] = {
+    Map[ApplicationAccessType, String] (
+      ApplicationAccessType.VIEW_APP -> (securityMgr.getViewAcls + " " +
+        securityMgr.getViewAclsGroups),
+      ApplicationAccessType.MODIFY_APP -> (securityMgr.getModifyAcls + " " +
+        securityMgr.getModifyAclsGroups)
+    )
+  }
+
+  /**
+   * Expand environment variable using Yarn API.
+   * If environment.$$() is implemented, return the result of it.
+   * Otherwise, return the result of environment.$()
+   * Note: $$() is added in Hadoop 2.4.
+   */
+  private lazy val expandMethod =
+    Try(classOf[Environment].getMethod("$$"))
+      .getOrElse(classOf[Environment].getMethod("$"))
+
+  def expandEnvironment(environment: Environment): String =
+    expandMethod.invoke(environment).asInstanceOf[String]
+
+  /**
+   * Get class path separator using Yarn API.
+   * If ApplicationConstants.CLASS_PATH_SEPARATOR is implemented, return it.
+   * Otherwise, return File.pathSeparator
+   * Note: CLASS_PATH_SEPARATOR is added in Hadoop 2.4.
+   */
+  private lazy val classPathSeparatorField =
+    Try(classOf[ApplicationConstants].getField("CLASS_PATH_SEPARATOR"))
+      .getOrElse(classOf[File].getField("pathSeparator"))
+
+  def getClassPathSeparator(): String = {
+    classPathSeparatorField.get(null).asInstanceOf[String]
+  }
+
+  /**
+   * Getting the initial target number of executors depends on whether dynamic allocation is
+   * enabled.
+   * If not using dynamic allocation it gets the number of executors requested by the user.
+   */
+  def getInitialTargetExecutorNumber(
+      conf: SparkConf,
+      numExecutors: Int = DEFAULT_NUMBER_EXECUTORS): Int = {
+    if (Utils.isDynamicAllocationEnabled(conf)) {
+      val minNumExecutors = conf.get(DYN_ALLOCATION_MIN_EXECUTORS)
+      val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf)
+      val maxNumExecutors = conf.get(DYN_ALLOCATION_MAX_EXECUTORS)
+      require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors,
+        s"initial executor number $initialNumExecutors must between min executor number " +
+          s"$minNumExecutors and max executor number $maxNumExecutors")
+
+      initialNumExecutors
+    } else {
+      val targetNumExecutors =
+        sys.env.get("SPARK_EXECUTOR_INSTANCES").map(_.toInt).getOrElse(numExecutors)
+      // System property can override environment variable.
+      conf.get(EXECUTOR_INSTANCES).getOrElse(targetNumExecutors)
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
new file mode 100644
index 0000000..666cb45
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
@@ -0,0 +1,347 @@
+/*
+ * 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 java.util.concurrent.TimeUnit
+
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+package object config {
+
+  /* Common app configuration. */
+
+  private[spark] val APPLICATION_TAGS = ConfigBuilder("spark.yarn.tags")
+    .doc("Comma-separated list of strings to pass through as YARN application tags appearing " +
+      "in YARN Application Reports, which can be used for filtering when querying YARN.")
+    .stringConf
+    .toSequence
+    .createOptional
+
+  private[spark] val AM_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS =
+    ConfigBuilder("spark.yarn.am.attemptFailuresValidityInterval")
+      .doc("Interval after which AM failures will be considered independent and " +
+        "not accumulate towards the attempt count.")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createOptional
+
+  private[spark] val AM_PORT =
+    ConfigBuilder("spark.yarn.am.port")
+      .intConf
+      .createWithDefault(0)
+
+  private[spark] val EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS =
+    ConfigBuilder("spark.yarn.executor.failuresValidityInterval")
+      .doc("Interval after which Executor failures will be considered independent and not " +
+        "accumulate towards the attempt count.")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createOptional
+
+  private[spark] val MAX_APP_ATTEMPTS = ConfigBuilder("spark.yarn.maxAppAttempts")
+    .doc("Maximum number of AM attempts before failing the app.")
+    .intConf
+    .createOptional
+
+  private[spark] val USER_CLASS_PATH_FIRST = ConfigBuilder("spark.yarn.user.classpath.first")
+    .doc("Whether to place user jars in front of Spark's classpath.")
+    .booleanConf
+    .createWithDefault(false)
+
+  private[spark] val GATEWAY_ROOT_PATH = ConfigBuilder("spark.yarn.config.gatewayPath")
+    .doc("Root of configuration paths that is present on gateway nodes, and will be replaced " +
+      "with the corresponding path in cluster machines.")
+    .stringConf
+    .createWithDefault(null)
+
+  private[spark] val REPLACEMENT_ROOT_PATH = ConfigBuilder("spark.yarn.config.replacementPath")
+    .doc(s"Path to use as a replacement for ${GATEWAY_ROOT_PATH.key} when launching processes " +
+      "in the YARN cluster.")
+    .stringConf
+    .createWithDefault(null)
+
+  private[spark] val QUEUE_NAME = ConfigBuilder("spark.yarn.queue")
+    .stringConf
+    .createWithDefault("default")
+
+  private[spark] val HISTORY_SERVER_ADDRESS = ConfigBuilder("spark.yarn.historyServer.address")
+    .stringConf
+    .createOptional
+
+  /* File distribution. */
+
+  private[spark] val SPARK_ARCHIVE = ConfigBuilder("spark.yarn.archive")
+    .doc("Location of archive containing jars files with Spark classes.")
+    .stringConf
+    .createOptional
+
+  private[spark] val SPARK_JARS = ConfigBuilder("spark.yarn.jars")
+    .doc("Location of jars containing Spark classes.")
+    .stringConf
+    .toSequence
+    .createOptional
+
+  private[spark] val ARCHIVES_TO_DISTRIBUTE = ConfigBuilder("spark.yarn.dist.archives")
+    .stringConf
+    .toSequence
+    .createWithDefault(Nil)
+
+  private[spark] val FILES_TO_DISTRIBUTE = ConfigBuilder("spark.yarn.dist.files")
+    .stringConf
+    .toSequence
+    .createWithDefault(Nil)
+
+  private[spark] val JARS_TO_DISTRIBUTE = ConfigBuilder("spark.yarn.dist.jars")
+    .stringConf
+    .toSequence
+    .createWithDefault(Nil)
+
+  private[spark] val PRESERVE_STAGING_FILES = ConfigBuilder("spark.yarn.preserve.staging.files")
+    .doc("Whether to preserve temporary files created by the job in HDFS.")
+    .booleanConf
+    .createWithDefault(false)
+
+  private[spark] val STAGING_FILE_REPLICATION = ConfigBuilder("spark.yarn.submit.file.replication")
+    .doc("Replication factor for files uploaded by Spark to HDFS.")
+    .intConf
+    .createOptional
+
+  private[spark] val STAGING_DIR = ConfigBuilder("spark.yarn.stagingDir")
+    .doc("Staging directory used while submitting applications.")
+    .stringConf
+    .createOptional
+
+  /* Cluster-mode launcher configuration. */
+
+  private[spark] val WAIT_FOR_APP_COMPLETION = ConfigBuilder("spark.yarn.submit.waitAppCompletion")
+    .doc("In cluster mode, whether to wait for the application to finish before exiting the " +
+      "launcher process.")
+    .booleanConf
+    .createWithDefault(true)
+
+  private[spark] val REPORT_INTERVAL = ConfigBuilder("spark.yarn.report.interval")
+    .doc("Interval between reports of the current app status in cluster mode.")
+    .timeConf(TimeUnit.MILLISECONDS)
+    .createWithDefaultString("1s")
+
+  /* Shared Client-mode AM / Driver configuration. */
+
+  private[spark] val AM_MAX_WAIT_TIME = ConfigBuilder("spark.yarn.am.waitTime")
+    .timeConf(TimeUnit.MILLISECONDS)
+    .createWithDefaultString("100s")
+
+  private[spark] val AM_NODE_LABEL_EXPRESSION = ConfigBuilder("spark.yarn.am.nodeLabelExpression")
+    .doc("Node label expression for the AM.")
+    .stringConf
+    .createOptional
+
+  private[spark] val CONTAINER_LAUNCH_MAX_THREADS =
+    ConfigBuilder("spark.yarn.containerLauncherMaxThreads")
+      .intConf
+      .createWithDefault(25)
+
+  private[spark] val MAX_EXECUTOR_FAILURES = ConfigBuilder("spark.yarn.max.executor.failures")
+    .intConf
+    .createOptional
+
+  private[spark] val MAX_REPORTER_THREAD_FAILURES =
+    ConfigBuilder("spark.yarn.scheduler.reporterThread.maxFailures")
+      .intConf
+      .createWithDefault(5)
+
+  private[spark] val RM_HEARTBEAT_INTERVAL =
+    ConfigBuilder("spark.yarn.scheduler.heartbeat.interval-ms")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createWithDefaultString("3s")
+
+  private[spark] val INITIAL_HEARTBEAT_INTERVAL =
+    ConfigBuilder("spark.yarn.scheduler.initial-allocation.interval")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createWithDefaultString("200ms")
+
+  private[spark] val SCHEDULER_SERVICES = ConfigBuilder("spark.yarn.services")
+    .doc("A comma-separated list of class names of services to add to the scheduler.")
+    .stringConf
+    .toSequence
+    .createWithDefault(Nil)
+
+  /* Client-mode AM configuration. */
+
+  private[spark] val AM_CORES = ConfigBuilder("spark.yarn.am.cores")
+    .intConf
+    .createWithDefault(1)
+
+  private[spark] val AM_JAVA_OPTIONS = ConfigBuilder("spark.yarn.am.extraJavaOptions")
+    .doc("Extra Java options for the client-mode AM.")
+    .stringConf
+    .createOptional
+
+  private[spark] val AM_LIBRARY_PATH = ConfigBuilder("spark.yarn.am.extraLibraryPath")
+    .doc("Extra native library path for the client-mode AM.")
+    .stringConf
+    .createOptional
+
+  private[spark] val AM_MEMORY_OVERHEAD = ConfigBuilder("spark.yarn.am.memoryOverhead")
+    .bytesConf(ByteUnit.MiB)
+    .createOptional
+
+  private[spark] val AM_MEMORY = ConfigBuilder("spark.yarn.am.memory")
+    .bytesConf(ByteUnit.MiB)
+    .createWithDefaultString("512m")
+
+  /* Driver configuration. */
+
+  private[spark] val DRIVER_CORES = ConfigBuilder("spark.driver.cores")
+    .intConf
+    .createWithDefault(1)
+
+  private[spark] val DRIVER_MEMORY_OVERHEAD = ConfigBuilder("spark.yarn.driver.memoryOverhead")
+    .bytesConf(ByteUnit.MiB)
+    .createOptional
+
+  /* Executor configuration. */
+
+  private[spark] val EXECUTOR_CORES = ConfigBuilder("spark.executor.cores")
+    .intConf
+    .createWithDefault(1)
+
+  private[spark] val EXECUTOR_MEMORY_OVERHEAD = ConfigBuilder("spark.yarn.executor.memoryOverhead")
+    .bytesConf(ByteUnit.MiB)
+    .createOptional
+
+  private[spark] val EXECUTOR_NODE_LABEL_EXPRESSION =
+    ConfigBuilder("spark.yarn.executor.nodeLabelExpression")
+      .doc("Node label expression for executors.")
+      .stringConf
+      .createOptional
+
+  /* Security configuration. */
+
+  private[spark] val CREDENTIAL_FILE_MAX_COUNT =
+    ConfigBuilder("spark.yarn.credentials.file.retention.count")
+      .intConf
+      .createWithDefault(5)
+
+  private[spark] val CREDENTIALS_FILE_MAX_RETENTION =
+    ConfigBuilder("spark.yarn.credentials.file.retention.days")
+      .intConf
+      .createWithDefault(5)
+
+  private[spark] val NAMENODES_TO_ACCESS = ConfigBuilder("spark.yarn.access.namenodes")
+    .doc("Extra NameNode URLs for which to request delegation tokens. The NameNode that hosts " +
+      "fs.defaultFS does not need to be listed here.")
+    .stringConf
+    .toSequence
+    .createWithDefault(Nil)
+
+  /* Rolled log aggregation configuration. */
+
+  private[spark] val ROLLED_LOG_INCLUDE_PATTERN =
+    ConfigBuilder("spark.yarn.rolledLog.includePattern")
+      .doc("Java Regex to filter the log files which match the defined include pattern and those " +
+        "log files will be aggregated in a rolling fashion.")
+      .stringConf
+      .createOptional
+
+  private[spark] val ROLLED_LOG_EXCLUDE_PATTERN =
+    ConfigBuilder("spark.yarn.rolledLog.excludePattern")
+      .doc("Java Regex to filter the log files which match the defined exclude pattern and those " +
+        "log files will not be aggregated in a rolling fashion.")
+      .stringConf
+      .createOptional
+
+  /* Private configs. */
+
+  private[spark] val CREDENTIALS_FILE_PATH = ConfigBuilder("spark.yarn.credentials.file")
+    .internal()
+    .stringConf
+    .createWithDefault(null)
+
+  // Internal config to propagate the location of the user's jar to the driver/executors
+  private[spark] val APP_JAR = ConfigBuilder("spark.yarn.user.jar")
+    .internal()
+    .stringConf
+    .createOptional
+
+  // Internal config to propagate the locations of any extra jars to add to the classpath
+  // of the executors
+  private[spark] val SECONDARY_JARS = ConfigBuilder("spark.yarn.secondary.jars")
+    .internal()
+    .stringConf
+    .toSequence
+    .createOptional
+
+  /* Configuration and cached file propagation. */
+
+  private[spark] val CACHED_FILES = ConfigBuilder("spark.yarn.cache.filenames")
+    .internal()
+    .stringConf
+    .toSequence
+    .createWithDefault(Nil)
+
+  private[spark] val CACHED_FILES_SIZES = ConfigBuilder("spark.yarn.cache.sizes")
+    .internal()
+    .longConf
+    .toSequence
+    .createWithDefault(Nil)
+
+  private[spark] val CACHED_FILES_TIMESTAMPS = ConfigBuilder("spark.yarn.cache.timestamps")
+    .internal()
+    .longConf
+    .toSequence
+    .createWithDefault(Nil)
+
+  private[spark] val CACHED_FILES_VISIBILITIES = ConfigBuilder("spark.yarn.cache.visibilities")
+    .internal()
+    .stringConf
+    .toSequence
+    .createWithDefault(Nil)
+
+  // Either "file" or "archive", for each file.
+  private[spark] val CACHED_FILES_TYPES = ConfigBuilder("spark.yarn.cache.types")
+    .internal()
+    .stringConf
+    .toSequence
+    .createWithDefault(Nil)
+
+  // The location of the conf archive in HDFS.
+  private[spark] val CACHED_CONF_ARCHIVE = ConfigBuilder("spark.yarn.cache.confArchive")
+    .internal()
+    .stringConf
+    .createOptional
+
+  private[spark] val CREDENTIALS_RENEWAL_TIME = ConfigBuilder("spark.yarn.credentials.renewalTime")
+    .internal()
+    .timeConf(TimeUnit.MILLISECONDS)
+    .createWithDefault(Long.MaxValue)
+
+  private[spark] val CREDENTIALS_UPDATE_TIME = ConfigBuilder("spark.yarn.credentials.updateTime")
+    .internal()
+    .timeConf(TimeUnit.MILLISECONDS)
+    .createWithDefault(Long.MaxValue)
+
+  // The list of cache-related config entries. This is used by Client and the AM to clean
+  // up the environment so that these settings do not appear on the web UI.
+  private[yarn] val CACHE_CONFIGS = Seq(
+    CACHED_FILES,
+    CACHED_FILES_SIZES,
+    CACHED_FILES_TIMESTAMPS,
+    CACHED_FILES_VISIBILITIES,
+    CACHED_FILES_TYPES,
+    CACHED_CONF_ARCHIVE)
+
+}


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


[03/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala
deleted file mode 100644
index c4c07b4..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala
+++ /dev/null
@@ -1,105 +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.deploy.yarn.security
-
-import java.util.ServiceLoader
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.security.Credentials
-
-import org.apache.spark.SparkConf
-import org.apache.spark.internal.Logging
-import org.apache.spark.util.Utils
-
-/**
- * A ConfigurableCredentialManager to manage all the registered credential providers and offer
- * APIs for other modules to obtain credentials as well as renewal time. By default
- * [[HDFSCredentialProvider]], [[HiveCredentialProvider]] and [[HBaseCredentialProvider]] will
- * be loaded in if not explicitly disabled, any plugged-in credential provider wants to be
- * managed by ConfigurableCredentialManager needs to implement [[ServiceCredentialProvider]]
- * interface and put into resources/META-INF/services to be loaded by ServiceLoader.
- *
- * Also each credential provider is controlled by
- * spark.yarn.security.credentials.{service}.enabled, it will not be loaded in if set to false.
- */
-private[yarn] final class ConfigurableCredentialManager(
-    sparkConf: SparkConf, hadoopConf: Configuration) extends Logging {
-  private val deprecatedProviderEnabledConfig = "spark.yarn.security.tokens.%s.enabled"
-  private val providerEnabledConfig = "spark.yarn.security.credentials.%s.enabled"
-
-  // Maintain all the registered credential providers
-  private val credentialProviders = {
-    val providers = ServiceLoader.load(classOf[ServiceCredentialProvider],
-      Utils.getContextOrSparkClassLoader).asScala
-
-    // Filter out credentials in which spark.yarn.security.credentials.{service}.enabled is false.
-    providers.filter { p =>
-      sparkConf.getOption(providerEnabledConfig.format(p.serviceName))
-        .orElse {
-          sparkConf.getOption(deprecatedProviderEnabledConfig.format(p.serviceName)).map { c =>
-            logWarning(s"${deprecatedProviderEnabledConfig.format(p.serviceName)} is deprecated, " +
-              s"using ${providerEnabledConfig.format(p.serviceName)} instead")
-            c
-          }
-        }.map(_.toBoolean).getOrElse(true)
-    }.map { p => (p.serviceName, p) }.toMap
-  }
-
-  /**
-   * Get credential provider for the specified service.
-   */
-  def getServiceCredentialProvider(service: String): Option[ServiceCredentialProvider] = {
-    credentialProviders.get(service)
-  }
-
-  /**
-   * Obtain credentials from all the registered providers.
-   * @return nearest time of next renewal, Long.MaxValue if all the credentials aren't renewable,
-   *         otherwise the nearest renewal time of any credentials will be returned.
-   */
-  def obtainCredentials(hadoopConf: Configuration, creds: Credentials): Long = {
-    credentialProviders.values.flatMap { provider =>
-      if (provider.credentialsRequired(hadoopConf)) {
-        provider.obtainCredentials(hadoopConf, sparkConf, creds)
-      } else {
-        logDebug(s"Service ${provider.serviceName} does not require a token." +
-          s" Check your configuration to see if security is disabled or not.")
-        None
-      }
-    }.foldLeft(Long.MaxValue)(math.min)
-  }
-
-  /**
-   * Create an [[AMCredentialRenewer]] instance, caller should be responsible to stop this
-   * instance when it is not used. AM will use it to renew credentials periodically.
-   */
-  def credentialRenewer(): AMCredentialRenewer = {
-    new AMCredentialRenewer(sparkConf, hadoopConf, this)
-  }
-
-  /**
-   * Create an [[CredentialUpdater]] instance, caller should be resposible to stop this intance
-   * when it is not used. Executors and driver (client mode) will use it to update credentials.
-   * periodically.
-   */
-  def credentialUpdater(): CredentialUpdater = {
-    new CredentialUpdater(sparkConf, hadoopConf, this)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala
deleted file mode 100644
index 5df4fbd..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala
+++ /dev/null
@@ -1,130 +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.deploy.yarn.security
-
-import java.util.concurrent.{Executors, TimeUnit}
-
-import scala.util.control.NonFatal
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileSystem, Path}
-import org.apache.hadoop.security.{Credentials, UserGroupInformation}
-
-import org.apache.spark.SparkConf
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.internal.Logging
-import org.apache.spark.util.{ThreadUtils, Utils}
-
-private[spark] class CredentialUpdater(
-    sparkConf: SparkConf,
-    hadoopConf: Configuration,
-    credentialManager: ConfigurableCredentialManager) extends Logging {
-
-  @volatile private var lastCredentialsFileSuffix = 0
-
-  private val credentialsFile = sparkConf.get(CREDENTIALS_FILE_PATH)
-  private val freshHadoopConf =
-    SparkHadoopUtil.get.getConfBypassingFSCache(
-      hadoopConf, new Path(credentialsFile).toUri.getScheme)
-
-  private val credentialUpdater =
-    Executors.newSingleThreadScheduledExecutor(
-      ThreadUtils.namedThreadFactory("Credential Refresh Thread"))
-
-  // This thread wakes up and picks up new credentials from HDFS, if any.
-  private val credentialUpdaterRunnable =
-    new Runnable {
-      override def run(): Unit = Utils.logUncaughtExceptions(updateCredentialsIfRequired())
-    }
-
-  /** Start the credential updater task */
-  def start(): Unit = {
-    val startTime = sparkConf.get(CREDENTIALS_RENEWAL_TIME)
-    val remainingTime = startTime - System.currentTimeMillis()
-    if (remainingTime <= 0) {
-      credentialUpdater.schedule(credentialUpdaterRunnable, 1, TimeUnit.MINUTES)
-    } else {
-      logInfo(s"Scheduling credentials refresh from HDFS in $remainingTime millis.")
-      credentialUpdater.schedule(credentialUpdaterRunnable, remainingTime, TimeUnit.MILLISECONDS)
-    }
-  }
-
-  private def updateCredentialsIfRequired(): Unit = {
-    val timeToNextUpdate = try {
-      val credentialsFilePath = new Path(credentialsFile)
-      val remoteFs = FileSystem.get(freshHadoopConf)
-      SparkHadoopUtil.get.listFilesSorted(
-        remoteFs, credentialsFilePath.getParent,
-        credentialsFilePath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
-        .lastOption.map { credentialsStatus =>
-          val suffix = SparkHadoopUtil.get.getSuffixForCredentialsPath(credentialsStatus.getPath)
-          if (suffix > lastCredentialsFileSuffix) {
-            logInfo("Reading new credentials from " + credentialsStatus.getPath)
-            val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath)
-            lastCredentialsFileSuffix = suffix
-            UserGroupInformation.getCurrentUser.addCredentials(newCredentials)
-            logInfo("Credentials updated from credentials file.")
-
-            val remainingTime = getTimeOfNextUpdateFromFileName(credentialsStatus.getPath)
-              - System.currentTimeMillis()
-            if (remainingTime <= 0) TimeUnit.MINUTES.toMillis(1) else remainingTime
-          } else {
-            // If current credential file is older than expected, sleep 1 hour and check again.
-            TimeUnit.HOURS.toMillis(1)
-          }
-      }.getOrElse {
-        // Wait for 1 minute to check again if there's no credential file currently
-        TimeUnit.MINUTES.toMillis(1)
-      }
-    } catch {
-      // Since the file may get deleted while we are reading it, catch the Exception and come
-      // back in an hour to try again
-      case NonFatal(e) =>
-        logWarning("Error while trying to update credentials, will try again in 1 hour", e)
-        TimeUnit.HOURS.toMillis(1)
-    }
-
-    credentialUpdater.schedule(
-      credentialUpdaterRunnable, timeToNextUpdate, TimeUnit.MILLISECONDS)
-  }
-
-  private def getCredentialsFromHDFSFile(remoteFs: FileSystem, tokenPath: Path): Credentials = {
-    val stream = remoteFs.open(tokenPath)
-    try {
-      val newCredentials = new Credentials()
-      newCredentials.readTokenStorageStream(stream)
-      newCredentials
-    } finally {
-      stream.close()
-    }
-  }
-
-  private def getTimeOfNextUpdateFromFileName(credentialsPath: Path): Long = {
-    val name = credentialsPath.getName
-    val index = name.lastIndexOf(SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM)
-    val slice = name.substring(0, index)
-    val last2index = slice.lastIndexOf(SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM)
-    name.substring(last2index + 1, index).toLong
-  }
-
-  def stop(): Unit = {
-    credentialUpdater.shutdown()
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala
deleted file mode 100644
index 5571df0..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala
+++ /dev/null
@@ -1,74 +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.deploy.yarn.security
-
-import scala.reflect.runtime.universe
-import scala.util.control.NonFatal
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.security.Credentials
-import org.apache.hadoop.security.token.{Token, TokenIdentifier}
-
-import org.apache.spark.SparkConf
-import org.apache.spark.internal.Logging
-
-private[security] class HBaseCredentialProvider extends ServiceCredentialProvider with Logging {
-
-  override def serviceName: String = "hbase"
-
-  override def obtainCredentials(
-      hadoopConf: Configuration,
-      sparkConf: SparkConf,
-      creds: Credentials): Option[Long] = {
-    try {
-      val mirror = universe.runtimeMirror(getClass.getClassLoader)
-      val obtainToken = mirror.classLoader.
-        loadClass("org.apache.hadoop.hbase.security.token.TokenUtil").
-        getMethod("obtainToken", classOf[Configuration])
-
-      logDebug("Attempting to fetch HBase security token.")
-      val token = obtainToken.invoke(null, hbaseConf(hadoopConf))
-        .asInstanceOf[Token[_ <: TokenIdentifier]]
-      logInfo(s"Get token from HBase: ${token.toString}")
-      creds.addToken(token.getService, token)
-    } catch {
-      case NonFatal(e) =>
-        logDebug(s"Failed to get token from service $serviceName", e)
-    }
-
-    None
-  }
-
-  override def credentialsRequired(hadoopConf: Configuration): Boolean = {
-    hbaseConf(hadoopConf).get("hbase.security.authentication") == "kerberos"
-  }
-
-  private def hbaseConf(conf: Configuration): Configuration = {
-    try {
-      val mirror = universe.runtimeMirror(getClass.getClassLoader)
-      val confCreate = mirror.classLoader.
-        loadClass("org.apache.hadoop.hbase.HBaseConfiguration").
-        getMethod("create", classOf[Configuration])
-      confCreate.invoke(null, conf).asInstanceOf[Configuration]
-    } catch {
-      case NonFatal(e) =>
-        logDebug("Fail to invoke HBaseConfiguration", e)
-        conf
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProvider.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProvider.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProvider.scala
deleted file mode 100644
index 8d06d73..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProvider.scala
+++ /dev/null
@@ -1,110 +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.deploy.yarn.security
-
-import java.io.{ByteArrayInputStream, DataInputStream}
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileSystem, Path}
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier
-import org.apache.hadoop.mapred.Master
-import org.apache.hadoop.security.Credentials
-
-import org.apache.spark.{SparkConf, SparkException}
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.internal.Logging
-import org.apache.spark.internal.config._
-
-private[security] class HDFSCredentialProvider extends ServiceCredentialProvider with Logging {
-  // Token renewal interval, this value will be set in the first call,
-  // if None means no token renewer specified, so cannot get token renewal interval.
-  private var tokenRenewalInterval: Option[Long] = null
-
-  override val serviceName: String = "hdfs"
-
-  override def obtainCredentials(
-      hadoopConf: Configuration,
-      sparkConf: SparkConf,
-      creds: Credentials): Option[Long] = {
-    // NameNode to access, used to get tokens from different FileSystems
-    nnsToAccess(hadoopConf, sparkConf).foreach { dst =>
-      val dstFs = dst.getFileSystem(hadoopConf)
-      logInfo("getting token for namenode: " + dst)
-      dstFs.addDelegationTokens(getTokenRenewer(hadoopConf), creds)
-    }
-
-    // Get the token renewal interval if it is not set. It will only be called once.
-    if (tokenRenewalInterval == null) {
-      tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf)
-    }
-
-    // Get the time of next renewal.
-    tokenRenewalInterval.map { interval =>
-      creds.getAllTokens.asScala
-        .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND)
-        .map { t =>
-          val identifier = new DelegationTokenIdentifier()
-          identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier)))
-          identifier.getIssueDate + interval
-      }.foldLeft(0L)(math.max)
-    }
-  }
-
-  private def getTokenRenewalInterval(
-      hadoopConf: Configuration, sparkConf: SparkConf): Option[Long] = {
-    // We cannot use the tokens generated with renewer yarn. Trying to renew
-    // those will fail with an access control issue. So create new tokens with the logged in
-    // user as renewer.
-    sparkConf.get(PRINCIPAL).map { renewer =>
-      val creds = new Credentials()
-      nnsToAccess(hadoopConf, sparkConf).foreach { dst =>
-        val dstFs = dst.getFileSystem(hadoopConf)
-        dstFs.addDelegationTokens(renewer, creds)
-      }
-      val t = creds.getAllTokens.asScala
-        .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND)
-        .head
-      val newExpiration = t.renew(hadoopConf)
-      val identifier = new DelegationTokenIdentifier()
-      identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier)))
-      val interval = newExpiration - identifier.getIssueDate
-      logInfo(s"Renewal Interval is $interval")
-      interval
-    }
-  }
-
-  private def getTokenRenewer(conf: Configuration): String = {
-    val delegTokenRenewer = Master.getMasterPrincipal(conf)
-    logDebug("delegation token renewer is: " + delegTokenRenewer)
-    if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
-      val errorMessage = "Can't get Master Kerberos principal for use as renewer"
-      logError(errorMessage)
-      throw new SparkException(errorMessage)
-    }
-
-    delegTokenRenewer
-  }
-
-  private def nnsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = {
-    sparkConf.get(NAMENODES_TO_ACCESS).map(new Path(_)).toSet +
-      sparkConf.get(STAGING_DIR).map(new Path(_))
-        .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala
deleted file mode 100644
index 16d8fc3..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala
+++ /dev/null
@@ -1,129 +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.deploy.yarn.security
-
-import java.lang.reflect.UndeclaredThrowableException
-import java.security.PrivilegedExceptionAction
-
-import scala.reflect.runtime.universe
-import scala.util.control.NonFatal
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier
-import org.apache.hadoop.io.Text
-import org.apache.hadoop.security.{Credentials, UserGroupInformation}
-import org.apache.hadoop.security.token.Token
-
-import org.apache.spark.SparkConf
-import org.apache.spark.internal.Logging
-import org.apache.spark.util.Utils
-
-private[security] class HiveCredentialProvider extends ServiceCredentialProvider with Logging {
-
-  override def serviceName: String = "hive"
-
-  private def hiveConf(hadoopConf: Configuration): Configuration = {
-    try {
-      val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
-      // the hive configuration class is a subclass of Hadoop Configuration, so can be cast down
-      // to a Configuration and used without reflection
-      val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf")
-      // using the (Configuration, Class) constructor allows the current configuration to be
-      // included in the hive config.
-      val ctor = hiveConfClass.getDeclaredConstructor(classOf[Configuration],
-        classOf[Object].getClass)
-      ctor.newInstance(hadoopConf, hiveConfClass).asInstanceOf[Configuration]
-    } catch {
-      case NonFatal(e) =>
-        logDebug("Fail to create Hive Configuration", e)
-        hadoopConf
-    }
-  }
-
-  override def credentialsRequired(hadoopConf: Configuration): Boolean = {
-    UserGroupInformation.isSecurityEnabled &&
-      hiveConf(hadoopConf).getTrimmed("hive.metastore.uris", "").nonEmpty
-  }
-
-  override def obtainCredentials(
-      hadoopConf: Configuration,
-      sparkConf: SparkConf,
-      creds: Credentials): Option[Long] = {
-    val conf = hiveConf(hadoopConf)
-
-    val principalKey = "hive.metastore.kerberos.principal"
-    val principal = conf.getTrimmed(principalKey, "")
-    require(principal.nonEmpty, s"Hive principal $principalKey undefined")
-    val metastoreUri = conf.getTrimmed("hive.metastore.uris", "")
-    require(metastoreUri.nonEmpty, "Hive metastore uri undefined")
-
-    val currentUser = UserGroupInformation.getCurrentUser()
-    logDebug(s"Getting Hive delegation token for ${currentUser.getUserName()} against " +
-      s"$principal at $metastoreUri")
-
-    val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
-    val hiveClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.ql.metadata.Hive")
-    val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf")
-    val closeCurrent = hiveClass.getMethod("closeCurrent")
-
-    try {
-      // get all the instance methods before invoking any
-      val getDelegationToken = hiveClass.getMethod("getDelegationToken",
-        classOf[String], classOf[String])
-      val getHive = hiveClass.getMethod("get", hiveConfClass)
-
-      doAsRealUser {
-        val hive = getHive.invoke(null, conf)
-        val tokenStr = getDelegationToken.invoke(hive, currentUser.getUserName(), principal)
-          .asInstanceOf[String]
-        val hive2Token = new Token[DelegationTokenIdentifier]()
-        hive2Token.decodeFromUrlString(tokenStr)
-        logInfo(s"Get Token from hive metastore: ${hive2Token.toString}")
-        creds.addToken(new Text("hive.server2.delegation.token"), hive2Token)
-      }
-    } catch {
-      case NonFatal(e) =>
-        logDebug(s"Fail to get token from service $serviceName", e)
-    } finally {
-      Utils.tryLogNonFatalError {
-        closeCurrent.invoke(null)
-      }
-    }
-
-    None
-  }
-
-  /**
-   * Run some code as the real logged in user (which may differ from the current user, for
-   * example, when using proxying).
-   */
-  private def doAsRealUser[T](fn: => T): T = {
-    val currentUser = UserGroupInformation.getCurrentUser()
-    val realUser = Option(currentUser.getRealUser()).getOrElse(currentUser)
-
-   // For some reason the Scala-generated anonymous class ends up causing an
-   // UndeclaredThrowableException, even if you annotate the method with @throws.
-   try {
-      realUser.doAs(new PrivilegedExceptionAction[T]() {
-        override def run(): T = fn
-      })
-    } catch {
-      case e: UndeclaredThrowableException => throw Option(e.getCause()).getOrElse(e)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala
deleted file mode 100644
index 4e3fcce..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala
+++ /dev/null
@@ -1,57 +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.deploy.yarn.security
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.security.{Credentials, UserGroupInformation}
-
-import org.apache.spark.SparkConf
-
-/**
- * A credential provider for a service. User must implement this if they need to access a
- * secure service from Spark.
- */
-trait ServiceCredentialProvider {
-
-  /**
-   * Name of the service to provide credentials. This name should unique, Spark internally will
-   * use this name to differentiate credential provider.
-   */
-  def serviceName: String
-
-  /**
-   * To decide whether credential is required for this service. By default it based on whether
-   * Hadoop security is enabled.
-   */
-  def credentialsRequired(hadoopConf: Configuration): Boolean = {
-    UserGroupInformation.isSecurityEnabled
-  }
-
-  /**
-   * Obtain credentials for this service and get the time of the next renewal.
-   * @param hadoopConf Configuration of current Hadoop Compatible system.
-   * @param sparkConf Spark configuration.
-   * @param creds Credentials to add tokens and security keys to.
-   * @return If this Credential is renewable and can be renewed, return the time of the next
-   *         renewal, otherwise None should be returned.
-   */
-  def obtainCredentials(
-      hadoopConf: Configuration,
-      sparkConf: SparkConf,
-      creds: Credentials): Option[Long]
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala b/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
deleted file mode 100644
index 6c3556a..0000000
--- a/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.launcher
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ListBuffer
-import scala.util.Properties
-
-/**
- * Exposes methods from the launcher library that are used by the YARN backend.
- */
-private[spark] object YarnCommandBuilderUtils {
-
-  def quoteForBatchScript(arg: String): String = {
-    CommandBuilderUtils.quoteForBatchScript(arg)
-  }
-
-  def findJarsDir(sparkHome: String): String = {
-    val scalaVer = Properties.versionNumberString
-      .split("\\.")
-      .take(2)
-      .mkString(".")
-    CommandBuilderUtils.findJarsDir(sparkHome, scalaVer, true)
-  }
-
-  /**
-   * Adds the perm gen configuration to the list of java options if needed and not yet added.
-   *
-   * Note that this method adds the option based on the local JVM version; if the node where
-   * the container is running has a different Java version, there's a risk that the option will
-   * not be added (e.g. if the AM is running Java 8 but the container's node is set up to use
-   * Java 7).
-   */
-  def addPermGenSizeOpt(args: ListBuffer[String]): Unit = {
-    CommandBuilderUtils.addPermGenSizeOpt(args.asJava)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala
deleted file mode 100644
index 4ed2852..0000000
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala
+++ /dev/null
@@ -1,143 +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 java.util.concurrent.atomic.AtomicBoolean
-
-import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
-
-import org.apache.spark.SparkContext
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.internal.Logging
-import org.apache.spark.util.Utils
-
-/**
- * An extension service that can be loaded into a Spark YARN scheduler.
- * A Service that can be started and stopped.
- *
- * 1. For implementations to be loadable by `SchedulerExtensionServices`,
- * they must provide an empty constructor.
- * 2. The `stop()` operation MUST be idempotent, and succeed even if `start()` was
- * never invoked.
- */
-trait SchedulerExtensionService {
-
-  /**
-   * Start the extension service. This should be a no-op if
-   * called more than once.
-   * @param binding binding to the spark application and YARN
-   */
-  def start(binding: SchedulerExtensionServiceBinding): Unit
-
-  /**
-   * Stop the service
-   * The `stop()` operation MUST be idempotent, and succeed even if `start()` was
-   * never invoked.
-   */
-  def stop(): Unit
-}
-
-/**
- * Binding information for a [[SchedulerExtensionService]].
- *
- * The attempt ID will be set if the service is started within a YARN application master;
- * there is then a different attempt ID for every time that AM is restarted.
- * When the service binding is instantiated in client mode, there's no attempt ID, as it lacks
- * this information.
- * @param sparkContext current spark context
- * @param applicationId YARN application ID
- * @param attemptId YARN attemptID. This will always be unset in client mode, and always set in
- *                  cluster mode.
- */
-case class SchedulerExtensionServiceBinding(
-    sparkContext: SparkContext,
-    applicationId: ApplicationId,
-    attemptId: Option[ApplicationAttemptId] = None)
-
-/**
- * Container for [[SchedulerExtensionService]] instances.
- *
- * Loads Extension Services from the configuration property
- * `"spark.yarn.services"`, instantiates and starts them.
- * When stopped, it stops all child entries.
- *
- * The order in which child extension services are started and stopped
- * is undefined.
- */
-private[spark] class SchedulerExtensionServices extends SchedulerExtensionService
-    with Logging {
-  private var serviceOption: Option[String] = None
-  private var services: List[SchedulerExtensionService] = Nil
-  private val started = new AtomicBoolean(false)
-  private var binding: SchedulerExtensionServiceBinding = _
-
-  /**
-   * Binding operation will load the named services and call bind on them too; the
-   * entire set of services are then ready for `init()` and `start()` calls.
-   *
-   * @param binding binding to the spark application and YARN
-   */
-  def start(binding: SchedulerExtensionServiceBinding): Unit = {
-    if (started.getAndSet(true)) {
-      logWarning("Ignoring re-entrant start operation")
-      return
-    }
-    require(binding.sparkContext != null, "Null context parameter")
-    require(binding.applicationId != null, "Null appId parameter")
-    this.binding = binding
-    val sparkContext = binding.sparkContext
-    val appId = binding.applicationId
-    val attemptId = binding.attemptId
-    logInfo(s"Starting Yarn extension services with app $appId and attemptId $attemptId")
-
-    services = sparkContext.conf.get(SCHEDULER_SERVICES).map { sClass =>
-      val instance = Utils.classForName(sClass)
-        .newInstance()
-        .asInstanceOf[SchedulerExtensionService]
-      // bind this service
-      instance.start(binding)
-      logInfo(s"Service $sClass started")
-      instance
-    }.toList
-  }
-
-  /**
-   * Get the list of services.
-   *
-   * @return a list of services; Nil until the service is started
-   */
-  def getServices: List[SchedulerExtensionService] = services
-
-  /**
-   * Stop the services; idempotent.
-   *
-   */
-  override def stop(): Unit = {
-    if (started.getAndSet(false)) {
-      logInfo(s"Stopping $this")
-      services.foreach { s =>
-        Utils.tryLogNonFatalError(s.stop())
-      }
-    }
-  }
-
-  override def toString(): String = s"""SchedulerExtensionServices
-    |(serviceOption=$serviceOption,
-    | services=$services,
-    | started=$started)""".stripMargin
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
deleted file mode 100644
index 60da356..0000000
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
+++ /dev/null
@@ -1,157 +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.mutable.ArrayBuffer
-
-import org.apache.hadoop.yarn.api.records.YarnApplicationState
-
-import org.apache.spark.{SparkContext, SparkException}
-import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnSparkHadoopUtil}
-import org.apache.spark.internal.Logging
-import org.apache.spark.launcher.SparkAppHandle
-import org.apache.spark.scheduler.TaskSchedulerImpl
-
-private[spark] class YarnClientSchedulerBackend(
-    scheduler: TaskSchedulerImpl,
-    sc: SparkContext)
-  extends YarnSchedulerBackend(scheduler, sc)
-  with Logging {
-
-  private var client: Client = null
-  private var monitorThread: MonitorThread = null
-
-  /**
-   * Create a Yarn client to submit an application to the ResourceManager.
-   * This waits until the application is running.
-   */
-  override def start() {
-    val driverHost = conf.get("spark.driver.host")
-    val driverPort = conf.get("spark.driver.port")
-    val hostport = driverHost + ":" + driverPort
-    sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.webUrl) }
-
-    val argsArrayBuf = new ArrayBuffer[String]()
-    argsArrayBuf += ("--arg", hostport)
-
-    logDebug("ClientArguments called with: " + argsArrayBuf.mkString(" "))
-    val args = new ClientArguments(argsArrayBuf.toArray)
-    totalExpectedExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(conf)
-    client = new Client(args, conf)
-    bindToYarn(client.submitApplication(), None)
-
-    // SPARK-8687: Ensure all necessary properties have already been set before
-    // we initialize our driver scheduler backend, which serves these properties
-    // to the executors
-    super.start()
-    waitForApplication()
-
-    // SPARK-8851: In yarn-client mode, the AM still does the credentials refresh. The driver
-    // reads the credentials from HDFS, just like the executors and updates its own credentials
-    // cache.
-    if (conf.contains("spark.yarn.credentials.file")) {
-      YarnSparkHadoopUtil.get.startCredentialUpdater(conf)
-    }
-    monitorThread = asyncMonitorApplication()
-    monitorThread.start()
-  }
-
-  /**
-   * Report the state of the application until it is running.
-   * If the application has finished, failed or been killed in the process, throw an exception.
-   * This assumes both `client` and `appId` have already been set.
-   */
-  private def waitForApplication(): Unit = {
-    assert(client != null && appId.isDefined, "Application has not been submitted yet!")
-    val (state, _) = client.monitorApplication(appId.get, returnOnRunning = true) // blocking
-    if (state == YarnApplicationState.FINISHED ||
-      state == YarnApplicationState.FAILED ||
-      state == YarnApplicationState.KILLED) {
-      throw new SparkException("Yarn application has already ended! " +
-        "It might have been killed or unable to launch application master.")
-    }
-    if (state == YarnApplicationState.RUNNING) {
-      logInfo(s"Application ${appId.get} has started running.")
-    }
-  }
-
-  /**
-   * We create this class for SPARK-9519. Basically when we interrupt the monitor thread it's
-   * because the SparkContext is being shut down(sc.stop() called by user code), but if
-   * monitorApplication return, it means the Yarn application finished before sc.stop() was called,
-   * which means we should call sc.stop() here, and we don't allow the monitor to be interrupted
-   * before SparkContext stops successfully.
-   */
-  private class MonitorThread extends Thread {
-    private var allowInterrupt = true
-
-    override def run() {
-      try {
-        val (state, _) = client.monitorApplication(appId.get, logApplicationReport = false)
-        logError(s"Yarn application has already exited with state $state!")
-        allowInterrupt = false
-        sc.stop()
-      } catch {
-        case e: InterruptedException => logInfo("Interrupting monitor thread")
-      }
-    }
-
-    def stopMonitor(): Unit = {
-      if (allowInterrupt) {
-        this.interrupt()
-      }
-    }
-  }
-
-  /**
-   * Monitor the application state in a separate thread.
-   * If the application has exited for any reason, stop the SparkContext.
-   * This assumes both `client` and `appId` have already been set.
-   */
-  private def asyncMonitorApplication(): MonitorThread = {
-    assert(client != null && appId.isDefined, "Application has not been submitted yet!")
-    val t = new MonitorThread
-    t.setName("Yarn application state monitor")
-    t.setDaemon(true)
-    t
-  }
-
-  /**
-   * Stop the scheduler. This assumes `start()` has already been called.
-   */
-  override def stop() {
-    assert(client != null, "Attempted to stop this scheduler before starting it!")
-    if (monitorThread != null) {
-      monitorThread.stopMonitor()
-    }
-
-    // Report a final state to the launcher if one is connected. This is needed since in client
-    // mode this backend doesn't let the app monitor loop run to completion, so it does not report
-    // the final state itself.
-    //
-    // Note: there's not enough information at this point to provide a better final state,
-    // so assume the application was successful.
-    client.reportLauncherState(SparkAppHandle.State.FINISHED)
-
-    super.stop()
-    YarnSparkHadoopUtil.get.stopCredentialUpdater()
-    client.stop()
-    logInfo("Stopped")
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala
deleted file mode 100644
index 64cd1bd..0000000
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala
+++ /dev/null
@@ -1,56 +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 org.apache.spark.{SparkContext, SparkException}
-import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
-
-/**
- * Cluster Manager for creation of Yarn scheduler and backend
- */
-private[spark] class YarnClusterManager extends ExternalClusterManager {
-
-  override def canCreate(masterURL: String): Boolean = {
-    masterURL == "yarn"
-  }
-
-  override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = {
-    sc.deployMode match {
-      case "cluster" => new YarnClusterScheduler(sc)
-      case "client" => new YarnScheduler(sc)
-      case _ => throw new SparkException(s"Unknown deploy mode '${sc.deployMode}' for Yarn")
-    }
-  }
-
-  override def createSchedulerBackend(sc: SparkContext,
-      masterURL: String,
-      scheduler: TaskScheduler): SchedulerBackend = {
-    sc.deployMode match {
-      case "cluster" =>
-        new YarnClusterSchedulerBackend(scheduler.asInstanceOf[TaskSchedulerImpl], sc)
-      case "client" =>
-        new YarnClientSchedulerBackend(scheduler.asInstanceOf[TaskSchedulerImpl], sc)
-      case  _ =>
-        throw new SparkException(s"Unknown deploy mode '${sc.deployMode}' for Yarn")
-    }
-  }
-
-  override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = {
-    scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
deleted file mode 100644
index 96c9151..0000000
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
+++ /dev/null
@@ -1,37 +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 org.apache.spark._
-import org.apache.spark.deploy.yarn.ApplicationMaster
-
-/**
- * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of
- * ApplicationMaster, etc is done
- */
-private[spark] class YarnClusterScheduler(sc: SparkContext) extends YarnScheduler(sc) {
-
-  logInfo("Created YarnClusterScheduler")
-
-  override def postStartHook() {
-    ApplicationMaster.sparkContextInitialized(sc)
-    super.postStartHook()
-    logInfo("YarnClusterScheduler.postStartHook done")
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
deleted file mode 100644
index 4f3d5eb..0000000
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
+++ /dev/null
@@ -1,67 +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 org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-
-import org.apache.spark.SparkContext
-import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnSparkHadoopUtil}
-import org.apache.spark.scheduler.TaskSchedulerImpl
-import org.apache.spark.util.Utils
-
-private[spark] class YarnClusterSchedulerBackend(
-    scheduler: TaskSchedulerImpl,
-    sc: SparkContext)
-  extends YarnSchedulerBackend(scheduler, sc) {
-
-  override def start() {
-    val attemptId = ApplicationMaster.getAttemptId
-    bindToYarn(attemptId.getApplicationId(), Some(attemptId))
-    super.start()
-    totalExpectedExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sc.conf)
-  }
-
-  override def getDriverLogUrls: Option[Map[String, String]] = {
-    var driverLogs: Option[Map[String, String]] = None
-    try {
-      val yarnConf = new YarnConfiguration(sc.hadoopConfiguration)
-      val containerId = YarnSparkHadoopUtil.get.getContainerId
-
-      val httpAddress = System.getenv(Environment.NM_HOST.name()) +
-        ":" + System.getenv(Environment.NM_HTTP_PORT.name())
-      // lookup appropriate http scheme for container log urls
-      val yarnHttpPolicy = yarnConf.get(
-        YarnConfiguration.YARN_HTTP_POLICY_KEY,
-        YarnConfiguration.YARN_HTTP_POLICY_DEFAULT
-      )
-      val user = Utils.getCurrentUserName()
-      val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://"
-      val baseUrl = s"$httpScheme$httpAddress/node/containerlogs/$containerId/$user"
-      logDebug(s"Base URL for logs: $baseUrl")
-      driverLogs = Some(Map(
-        "stdout" -> s"$baseUrl/stdout?start=-4096",
-        "stderr" -> s"$baseUrl/stderr?start=-4096"))
-    } catch {
-      case e: Exception =>
-        logInfo("Error while building AM log links, so AM" +
-          " logs link will not appear in application UI", e)
-    }
-    driverLogs
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala
deleted file mode 100644
index 0293821..0000000
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala
+++ /dev/null
@@ -1,39 +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 org.apache.hadoop.yarn.util.RackResolver
-import org.apache.log4j.{Level, Logger}
-
-import org.apache.spark._
-import org.apache.spark.scheduler.TaskSchedulerImpl
-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)
-  }
-
-  // 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)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
deleted file mode 100644
index 2f9ea19..0000000
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
+++ /dev/null
@@ -1,315 +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.concurrent.{ExecutionContext, Future}
-import scala.util.{Failure, Success}
-import scala.util.control.NonFatal
-
-import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
-
-import org.apache.spark.SparkContext
-import org.apache.spark.internal.Logging
-import org.apache.spark.rpc._
-import org.apache.spark.scheduler._
-import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
-import org.apache.spark.ui.JettyUtils
-import org.apache.spark.util.{RpcUtils, ThreadUtils}
-
-/**
- * Abstract Yarn scheduler backend that contains common logic
- * between the client and cluster Yarn scheduler backends.
- */
-private[spark] abstract class YarnSchedulerBackend(
-    scheduler: TaskSchedulerImpl,
-    sc: SparkContext)
-  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) {
-
-  override val minRegisteredRatio =
-    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
-      0.8
-    } else {
-      super.minRegisteredRatio
-    }
-
-  protected var totalExpectedExecutors = 0
-
-  private val yarnSchedulerEndpoint = new YarnSchedulerEndpoint(rpcEnv)
-
-  private val yarnSchedulerEndpointRef = rpcEnv.setupEndpoint(
-    YarnSchedulerBackend.ENDPOINT_NAME, yarnSchedulerEndpoint)
-
-  private implicit val askTimeout = RpcUtils.askRpcTimeout(sc.conf)
-
-  /** Application ID. */
-  protected var appId: Option[ApplicationId] = None
-
-  /** Attempt ID. This is unset for client-mode schedulers */
-  private var attemptId: Option[ApplicationAttemptId] = None
-
-  /** Scheduler extension services. */
-  private val services: SchedulerExtensionServices = new SchedulerExtensionServices()
-
-  // Flag to specify whether this schedulerBackend should be reset.
-  private var shouldResetOnAmRegister = false
-
-  /**
-   * Bind to YARN. This *must* be done before calling [[start()]].
-   *
-   * @param appId YARN application ID
-   * @param attemptId Optional YARN attempt ID
-   */
-  protected def bindToYarn(appId: ApplicationId, attemptId: Option[ApplicationAttemptId]): Unit = {
-    this.appId = Some(appId)
-    this.attemptId = attemptId
-  }
-
-  override def start() {
-    require(appId.isDefined, "application ID unset")
-    val binding = SchedulerExtensionServiceBinding(sc, appId.get, attemptId)
-    services.start(binding)
-    super.start()
-  }
-
-  override def stop(): Unit = {
-    try {
-      // SPARK-12009: To prevent Yarn allocator from requesting backup for the executors which
-      // was Stopped by SchedulerBackend.
-      requestTotalExecutors(0, 0, Map.empty)
-      super.stop()
-    } finally {
-      services.stop()
-    }
-  }
-
-  /**
-   * Get the attempt ID for this run, if the cluster manager supports multiple
-   * attempts. Applications run in client mode will not have attempt IDs.
-   * This attempt ID only includes attempt counter, like "1", "2".
-   *
-   * @return The application attempt id, if available.
-   */
-  override def applicationAttemptId(): Option[String] = {
-    attemptId.map(_.getAttemptId.toString)
-  }
-
-  /**
-   * Get an application ID associated with the job.
-   * This returns the string value of [[appId]] if set, otherwise
-   * the locally-generated ID from the superclass.
-   * @return The application ID
-   */
-  override def applicationId(): String = {
-    appId.map(_.toString).getOrElse {
-      logWarning("Application ID is not initialized yet.")
-      super.applicationId
-    }
-  }
-
-  /**
-   * Request executors from the ApplicationMaster by specifying the total number desired.
-   * This includes executors already pending or running.
-   */
-  override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = {
-    yarnSchedulerEndpointRef.ask[Boolean](
-      RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount))
-  }
-
-  /**
-   * Request that the ApplicationMaster kill the specified executors.
-   */
-  override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = {
-    yarnSchedulerEndpointRef.ask[Boolean](KillExecutors(executorIds))
-  }
-
-  override def sufficientResourcesRegistered(): Boolean = {
-    totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio
-  }
-
-  /**
-   * Add filters to the SparkUI.
-   */
-  private def addWebUIFilter(
-      filterName: String,
-      filterParams: Map[String, String],
-      proxyBase: String): Unit = {
-    if (proxyBase != null && proxyBase.nonEmpty) {
-      System.setProperty("spark.ui.proxyBase", proxyBase)
-    }
-
-    val hasFilter =
-      filterName != null && filterName.nonEmpty &&
-      filterParams != null && filterParams.nonEmpty
-    if (hasFilter) {
-      logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase")
-      conf.set("spark.ui.filters", filterName)
-      filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) }
-      scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) }
-    }
-  }
-
-  override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = {
-    new YarnDriverEndpoint(rpcEnv, properties)
-  }
-
-  /**
-   * Reset the state of SchedulerBackend to the initial state. This is happened when AM is failed
-   * and re-registered itself to driver after a failure. The stale state in driver should be
-   * cleaned.
-   */
-  override protected def reset(): Unit = {
-    super.reset()
-    sc.executorAllocationManager.foreach(_.reset())
-  }
-
-  /**
-   * Override the DriverEndpoint to add extra logic for the case when an executor is disconnected.
-   * This endpoint communicates with the executors and queries the AM for an executor's exit
-   * status when the executor is disconnected.
-   */
-  private class YarnDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)])
-      extends DriverEndpoint(rpcEnv, sparkProperties) {
-
-    /**
-     * When onDisconnected is received at the driver endpoint, the superclass DriverEndpoint
-     * handles it by assuming the Executor was lost for a bad reason and removes the executor
-     * immediately.
-     *
-     * In YARN's case however it is crucial to talk to the application master and ask why the
-     * executor had exited. If the executor exited for some reason unrelated to the running tasks
-     * (e.g., preemption), according to the application master, then we pass that information down
-     * to the TaskSetManager to inform the TaskSetManager that tasks on that lost executor should
-     * not count towards a job failure.
-     */
-    override def onDisconnected(rpcAddress: RpcAddress): Unit = {
-      addressToExecutorId.get(rpcAddress).foreach { executorId =>
-        if (disableExecutor(executorId)) {
-          yarnSchedulerEndpoint.handleExecutorDisconnectedFromDriver(executorId, rpcAddress)
-        }
-      }
-    }
-  }
-
-  /**
-   * An [[RpcEndpoint]] that communicates with the ApplicationMaster.
-   */
-  private class YarnSchedulerEndpoint(override val rpcEnv: RpcEnv)
-    extends ThreadSafeRpcEndpoint with Logging {
-    private var amEndpoint: Option[RpcEndpointRef] = None
-
-    private[YarnSchedulerBackend] def handleExecutorDisconnectedFromDriver(
-        executorId: String,
-        executorRpcAddress: RpcAddress): Unit = {
-      val removeExecutorMessage = amEndpoint match {
-        case Some(am) =>
-          val lossReasonRequest = GetExecutorLossReason(executorId)
-          am.ask[ExecutorLossReason](lossReasonRequest, askTimeout)
-            .map { reason => RemoveExecutor(executorId, reason) }(ThreadUtils.sameThread)
-            .recover {
-              case NonFatal(e) =>
-                logWarning(s"Attempted to get executor loss reason" +
-                  s" for executor id ${executorId} at RPC address ${executorRpcAddress}," +
-                  s" but got no response. Marking as slave lost.", e)
-                RemoveExecutor(executorId, SlaveLost())
-            }(ThreadUtils.sameThread)
-        case None =>
-          logWarning("Attempted to check for an executor loss reason" +
-            " before the AM has registered!")
-          Future.successful(RemoveExecutor(executorId, SlaveLost("AM is not yet registered.")))
-      }
-
-      removeExecutorMessage
-        .flatMap { message =>
-          driverEndpoint.ask[Boolean](message)
-        }(ThreadUtils.sameThread)
-        .onFailure {
-          case NonFatal(e) => logError(
-            s"Error requesting driver to remove executor $executorId after disconnection.", e)
-        }(ThreadUtils.sameThread)
-    }
-
-    override def receive: PartialFunction[Any, Unit] = {
-      case RegisterClusterManager(am) =>
-        logInfo(s"ApplicationMaster registered as $am")
-        amEndpoint = Option(am)
-        if (!shouldResetOnAmRegister) {
-          shouldResetOnAmRegister = true
-        } else {
-          // AM is already registered before, this potentially means that AM failed and
-          // a new one registered after the failure. This will only happen in yarn-client mode.
-          reset()
-        }
-
-      case AddWebUIFilter(filterName, filterParams, proxyBase) =>
-        addWebUIFilter(filterName, filterParams, proxyBase)
-
-      case r @ RemoveExecutor(executorId, reason) =>
-        logWarning(reason.toString)
-        driverEndpoint.ask[Boolean](r).onFailure {
-          case e =>
-            logError("Error requesting driver to remove executor" +
-              s" $executorId for reason $reason", e)
-        }(ThreadUtils.sameThread)
-    }
-
-
-    override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
-      case r: RequestExecutors =>
-        amEndpoint match {
-          case Some(am) =>
-            am.ask[Boolean](r).andThen {
-              case Success(b) => context.reply(b)
-              case Failure(NonFatal(e)) =>
-                logError(s"Sending $r to AM was unsuccessful", e)
-                context.sendFailure(e)
-            }(ThreadUtils.sameThread)
-          case None =>
-            logWarning("Attempted to request executors before the AM has registered!")
-            context.reply(false)
-        }
-
-      case k: KillExecutors =>
-        amEndpoint match {
-          case Some(am) =>
-            am.ask[Boolean](k).andThen {
-              case Success(b) => context.reply(b)
-              case Failure(NonFatal(e)) =>
-                logError(s"Sending $k to AM was unsuccessful", e)
-                context.sendFailure(e)
-            }(ThreadUtils.sameThread)
-          case None =>
-            logWarning("Attempted to kill executors before the AM has registered!")
-            context.reply(false)
-        }
-
-      case RetrieveLastAllocatedExecutorId =>
-        context.reply(currentExecutorIdCounter)
-    }
-
-    override def onDisconnected(remoteAddress: RpcAddress): Unit = {
-      if (amEndpoint.exists(_.address == remoteAddress)) {
-        logWarning(s"ApplicationMaster has disassociated: $remoteAddress")
-        amEndpoint = None
-      }
-    }
-  }
-}
-
-private[spark] object YarnSchedulerBackend {
-  val ENDPOINT_NAME = "YarnScheduler"
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
----------------------------------------------------------------------
diff --git a/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider b/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
deleted file mode 100644
index d0ef5ef..0000000
--- a/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
+++ /dev/null
@@ -1 +0,0 @@
-org.apache.spark.deploy.yarn.security.TestCredentialProvider

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/yarn/src/test/resources/log4j.properties b/yarn/src/test/resources/log4j.properties
deleted file mode 100644
index d13454d..0000000
--- a/yarn/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,31 +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.
-#
-
-# Set everything to be logged to the file target/unit-tests.log
-log4j.rootCategory=DEBUG, file
-log4j.appender.file=org.apache.log4j.FileAppender
-log4j.appender.file.append=true
-log4j.appender.file.file=target/unit-tests.log
-log4j.appender.file.layout=org.apache.log4j.PatternLayout
-log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
-
-# Ignore messages below warning level from a few verbose libraries.
-log4j.logger.com.sun.jersey=WARN
-log4j.logger.org.apache.hadoop=WARN
-log4j.logger.org.eclipse.jetty=WARN
-log4j.logger.org.mortbay=WARN
-log4j.logger.org.spark_project.jetty=WARN

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala
deleted file mode 100644
index 9c3b18e..0000000
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala
+++ /dev/null
@@ -1,241 +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.deploy.yarn
-
-import java.io.{File, FileOutputStream, OutputStreamWriter}
-import java.nio.charset.StandardCharsets
-import java.util.Properties
-import java.util.concurrent.TimeUnit
-
-import scala.collection.JavaConverters._
-import scala.concurrent.duration._
-import scala.language.postfixOps
-
-import com.google.common.io.Files
-import org.apache.commons.lang3.SerializationUtils
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.yarn.server.MiniYARNCluster
-import org.scalatest.{BeforeAndAfterAll, Matchers}
-import org.scalatest.concurrent.Eventually._
-
-import org.apache.spark._
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.internal.Logging
-import org.apache.spark.launcher._
-import org.apache.spark.util.Utils
-
-abstract class BaseYarnClusterSuite
-  extends SparkFunSuite with BeforeAndAfterAll with Matchers with Logging {
-
-  // log4j configuration for the YARN containers, so that their output is collected
-  // by YARN instead of trying to overwrite unit-tests.log.
-  protected val LOG4J_CONF = """
-    |log4j.rootCategory=DEBUG, console
-    |log4j.appender.console=org.apache.log4j.ConsoleAppender
-    |log4j.appender.console.target=System.err
-    |log4j.appender.console.layout=org.apache.log4j.PatternLayout
-    |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n
-    |log4j.logger.org.apache.hadoop=WARN
-    |log4j.logger.org.eclipse.jetty=WARN
-    |log4j.logger.org.mortbay=WARN
-    |log4j.logger.org.spark_project.jetty=WARN
-    """.stripMargin
-
-  private var yarnCluster: MiniYARNCluster = _
-  protected var tempDir: File = _
-  private var fakeSparkJar: File = _
-  protected var hadoopConfDir: File = _
-  private var logConfDir: File = _
-
-  var oldSystemProperties: Properties = null
-
-  def newYarnConfig(): YarnConfiguration
-
-  override def beforeAll() {
-    super.beforeAll()
-    oldSystemProperties = SerializationUtils.clone(System.getProperties)
-
-    tempDir = Utils.createTempDir()
-    logConfDir = new File(tempDir, "log4j")
-    logConfDir.mkdir()
-    System.setProperty("SPARK_YARN_MODE", "true")
-
-    val logConfFile = new File(logConfDir, "log4j.properties")
-    Files.write(LOG4J_CONF, logConfFile, StandardCharsets.UTF_8)
-
-    // Disable the disk utilization check to avoid the test hanging when people's disks are
-    // getting full.
-    val yarnConf = newYarnConfig()
-    yarnConf.set("yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage",
-      "100.0")
-
-    yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1)
-    yarnCluster.init(yarnConf)
-    yarnCluster.start()
-
-    // There's a race in MiniYARNCluster in which start() may return before the RM has updated
-    // its address in the configuration. You can see this in the logs by noticing that when
-    // MiniYARNCluster prints the address, it still has port "0" assigned, although later the
-    // test works sometimes:
-    //
-    //    INFO MiniYARNCluster: MiniYARN ResourceManager address: blah:0
-    //
-    // That log message prints the contents of the RM_ADDRESS config variable. If you check it
-    // later on, it looks something like this:
-    //
-    //    INFO YarnClusterSuite: RM address in configuration is blah:42631
-    //
-    // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't
-    // done so in a timely manner (defined to be 10 seconds).
-    val config = yarnCluster.getConfig()
-    val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10)
-    while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") {
-      if (System.currentTimeMillis() > deadline) {
-        throw new IllegalStateException("Timed out waiting for RM to come up.")
-      }
-      logDebug("RM address still not set in configuration, waiting...")
-      TimeUnit.MILLISECONDS.sleep(100)
-    }
-
-    logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}")
-
-    fakeSparkJar = File.createTempFile("sparkJar", null, tempDir)
-    hadoopConfDir = new File(tempDir, Client.LOCALIZED_CONF_DIR)
-    assert(hadoopConfDir.mkdir())
-    File.createTempFile("token", ".txt", hadoopConfDir)
-  }
-
-  override def afterAll() {
-    try {
-      yarnCluster.stop()
-    } finally {
-      System.setProperties(oldSystemProperties)
-      super.afterAll()
-    }
-  }
-
-  protected def runSpark(
-      clientMode: Boolean,
-      klass: String,
-      appArgs: Seq[String] = Nil,
-      sparkArgs: Seq[(String, String)] = Nil,
-      extraClassPath: Seq[String] = Nil,
-      extraJars: Seq[String] = Nil,
-      extraConf: Map[String, String] = Map(),
-      extraEnv: Map[String, String] = Map()): SparkAppHandle.State = {
-    val deployMode = if (clientMode) "client" else "cluster"
-    val propsFile = createConfFile(extraClassPath = extraClassPath, extraConf = extraConf)
-    val env = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath()) ++ extraEnv
-
-    val launcher = new SparkLauncher(env.asJava)
-    if (klass.endsWith(".py")) {
-      launcher.setAppResource(klass)
-    } else {
-      launcher.setMainClass(klass)
-      launcher.setAppResource(fakeSparkJar.getAbsolutePath())
-    }
-    launcher.setSparkHome(sys.props("spark.test.home"))
-      .setMaster("yarn")
-      .setDeployMode(deployMode)
-      .setConf("spark.executor.instances", "1")
-      .setPropertiesFile(propsFile)
-      .addAppArgs(appArgs.toArray: _*)
-
-    sparkArgs.foreach { case (name, value) =>
-      if (value != null) {
-        launcher.addSparkArg(name, value)
-      } else {
-        launcher.addSparkArg(name)
-      }
-    }
-    extraJars.foreach(launcher.addJar)
-
-    val handle = launcher.startApplication()
-    try {
-      eventually(timeout(2 minutes), interval(1 second)) {
-        assert(handle.getState().isFinal())
-      }
-    } finally {
-      handle.kill()
-    }
-
-    handle.getState()
-  }
-
-  /**
-   * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide
-   * any sort of error when the job process finishes successfully, but the job itself fails. So
-   * the tests enforce that something is written to a file after everything is ok to indicate
-   * that the job succeeded.
-   */
-  protected def checkResult(finalState: SparkAppHandle.State, result: File): Unit = {
-    checkResult(finalState, result, "success")
-  }
-
-  protected def checkResult(
-      finalState: SparkAppHandle.State,
-      result: File,
-      expected: String): Unit = {
-    finalState should be (SparkAppHandle.State.FINISHED)
-    val resultString = Files.toString(result, StandardCharsets.UTF_8)
-    resultString should be (expected)
-  }
-
-  protected def mainClassName(klass: Class[_]): String = {
-    klass.getName().stripSuffix("$")
-  }
-
-  protected def createConfFile(
-      extraClassPath: Seq[String] = Nil,
-      extraConf: Map[String, String] = Map()): String = {
-    val props = new Properties()
-    props.put(SPARK_JARS.key, "local:" + fakeSparkJar.getAbsolutePath())
-
-    val testClasspath = new TestClasspathBuilder()
-      .buildClassPath(
-        logConfDir.getAbsolutePath() +
-        File.pathSeparator +
-        extraClassPath.mkString(File.pathSeparator))
-      .asScala
-      .mkString(File.pathSeparator)
-
-    props.put("spark.driver.extraClassPath", testClasspath)
-    props.put("spark.executor.extraClassPath", testClasspath)
-
-    // SPARK-4267: make sure java options are propagated correctly.
-    props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"")
-    props.setProperty("spark.executor.extraJavaOptions", "-Dfoo=\"one two three\"")
-
-    yarnCluster.getConfig().asScala.foreach { e =>
-      props.setProperty("spark.hadoop." + e.getKey(), e.getValue())
-    }
-    sys.props.foreach { case (k, v) =>
-      if (k.startsWith("spark.")) {
-        props.setProperty(k, v)
-      }
-    }
-    extraConf.foreach { case (k, v) => props.setProperty(k, v) }
-
-    val propsFile = File.createTempFile("spark", ".properties", tempDir)
-    val writer = new OutputStreamWriter(new FileOutputStream(propsFile), StandardCharsets.UTF_8)
-    props.store(writer, "Spark properties.")
-    writer.close()
-    propsFile.getAbsolutePath()
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
deleted file mode 100644
index b696e08..0000000
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
+++ /dev/null
@@ -1,204 +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.deploy.yarn
-
-import java.net.URI
-
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.Map
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.FileStatus
-import org.apache.hadoop.fs.FileSystem
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.yarn.api.records.LocalResource
-import org.apache.hadoop.yarn.api.records.LocalResourceType
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
-import org.apache.hadoop.yarn.util.ConverterUtils
-import org.mockito.Mockito.when
-import org.scalatest.mock.MockitoSugar
-
-import org.apache.spark.{SparkConf, SparkFunSuite}
-import org.apache.spark.deploy.yarn.config._
-
-class ClientDistributedCacheManagerSuite extends SparkFunSuite with MockitoSugar {
-
-  class MockClientDistributedCacheManager extends ClientDistributedCacheManager {
-    override def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]):
-        LocalResourceVisibility = {
-      LocalResourceVisibility.PRIVATE
-    }
-  }
-
-  test("test getFileStatus empty") {
-    val distMgr = new ClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val uri = new URI("/tmp/testing")
-    when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus())
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    val stat = distMgr.getFileStatus(fs, uri, statCache)
-    assert(stat.getPath() === null)
-  }
-
-  test("test getFileStatus cached") {
-    val distMgr = new ClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val uri = new URI("/tmp/testing")
-    val realFileStatus = new FileStatus(10, false, 1, 1024, 10, 10, null, "testOwner",
-      null, new Path("/tmp/testing"))
-    when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus())
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus](uri -> realFileStatus)
-    val stat = distMgr.getFileStatus(fs, uri, statCache)
-    assert(stat.getPath().toString() === "/tmp/testing")
-  }
-
-  test("test addResource") {
-    val distMgr = new MockClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val conf = new Configuration()
-    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
-    val localResources = HashMap[String, LocalResource]()
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    when(fs.getFileStatus(destPath)).thenReturn(new FileStatus())
-
-    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, "link",
-      statCache, false)
-    val resource = localResources("link")
-    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
-    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
-    assert(resource.getTimestamp() === 0)
-    assert(resource.getSize() === 0)
-    assert(resource.getType() === LocalResourceType.FILE)
-
-    val sparkConf = new SparkConf(false)
-    distMgr.updateConfiguration(sparkConf)
-    assert(sparkConf.get(CACHED_FILES) === Seq("file:/foo.invalid.com:8080/tmp/testing#link"))
-    assert(sparkConf.get(CACHED_FILES_TIMESTAMPS) === Seq(0L))
-    assert(sparkConf.get(CACHED_FILES_SIZES) === Seq(0L))
-    assert(sparkConf.get(CACHED_FILES_VISIBILITIES) === Seq(LocalResourceVisibility.PRIVATE.name()))
-    assert(sparkConf.get(CACHED_FILES_TYPES) === Seq(LocalResourceType.FILE.name()))
-
-    // add another one and verify both there and order correct
-    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner",
-      null, new Path("/tmp/testing2"))
-    val destPath2 = new Path("file:///foo.invalid.com:8080/tmp/testing2")
-    when(fs.getFileStatus(destPath2)).thenReturn(realFileStatus)
-    distMgr.addResource(fs, conf, destPath2, localResources, LocalResourceType.FILE, "link2",
-      statCache, false)
-    val resource2 = localResources("link2")
-    assert(resource2.getVisibility() === LocalResourceVisibility.PRIVATE)
-    assert(ConverterUtils.getPathFromYarnURL(resource2.getResource()) === destPath2)
-    assert(resource2.getTimestamp() === 10)
-    assert(resource2.getSize() === 20)
-    assert(resource2.getType() === LocalResourceType.FILE)
-
-    val sparkConf2 = new SparkConf(false)
-    distMgr.updateConfiguration(sparkConf2)
-
-    val files = sparkConf2.get(CACHED_FILES)
-    val sizes = sparkConf2.get(CACHED_FILES_SIZES)
-    val timestamps = sparkConf2.get(CACHED_FILES_TIMESTAMPS)
-    val visibilities = sparkConf2.get(CACHED_FILES_VISIBILITIES)
-
-    assert(files(0) === "file:/foo.invalid.com:8080/tmp/testing#link")
-    assert(timestamps(0)  === 0)
-    assert(sizes(0)  === 0)
-    assert(visibilities(0) === LocalResourceVisibility.PRIVATE.name())
-
-    assert(files(1) === "file:/foo.invalid.com:8080/tmp/testing2#link2")
-    assert(timestamps(1)  === 10)
-    assert(sizes(1)  === 20)
-    assert(visibilities(1) === LocalResourceVisibility.PRIVATE.name())
-  }
-
-  test("test addResource link null") {
-    val distMgr = new MockClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val conf = new Configuration()
-    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
-    val localResources = HashMap[String, LocalResource]()
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    when(fs.getFileStatus(destPath)).thenReturn(new FileStatus())
-
-    intercept[Exception] {
-      distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, null,
-        statCache, false)
-    }
-    assert(localResources.get("link") === None)
-    assert(localResources.size === 0)
-  }
-
-  test("test addResource appmaster only") {
-    val distMgr = new MockClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val conf = new Configuration()
-    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
-    val localResources = HashMap[String, LocalResource]()
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner",
-      null, new Path("/tmp/testing"))
-    when(fs.getFileStatus(destPath)).thenReturn(realFileStatus)
-
-    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link",
-      statCache, true)
-    val resource = localResources("link")
-    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
-    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
-    assert(resource.getTimestamp() === 10)
-    assert(resource.getSize() === 20)
-    assert(resource.getType() === LocalResourceType.ARCHIVE)
-
-    val sparkConf = new SparkConf(false)
-    distMgr.updateConfiguration(sparkConf)
-    assert(sparkConf.get(CACHED_FILES) === Nil)
-    assert(sparkConf.get(CACHED_FILES_TIMESTAMPS) === Nil)
-    assert(sparkConf.get(CACHED_FILES_SIZES) === Nil)
-    assert(sparkConf.get(CACHED_FILES_VISIBILITIES) === Nil)
-    assert(sparkConf.get(CACHED_FILES_TYPES) === Nil)
-  }
-
-  test("test addResource archive") {
-    val distMgr = new MockClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val conf = new Configuration()
-    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
-    val localResources = HashMap[String, LocalResource]()
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner",
-      null, new Path("/tmp/testing"))
-    when(fs.getFileStatus(destPath)).thenReturn(realFileStatus)
-
-    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link",
-      statCache, false)
-    val resource = localResources("link")
-    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
-    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
-    assert(resource.getTimestamp() === 10)
-    assert(resource.getSize() === 20)
-    assert(resource.getType() === LocalResourceType.ARCHIVE)
-
-    val sparkConf = new SparkConf(false)
-    distMgr.updateConfiguration(sparkConf)
-    assert(sparkConf.get(CACHED_FILES) === Seq("file:/foo.invalid.com:8080/tmp/testing#link"))
-    assert(sparkConf.get(CACHED_FILES_SIZES) === Seq(20L))
-    assert(sparkConf.get(CACHED_FILES_TIMESTAMPS) === Seq(10L))
-    assert(sparkConf.get(CACHED_FILES_VISIBILITIES) === Seq(LocalResourceVisibility.PRIVATE.name()))
-    assert(sparkConf.get(CACHED_FILES_TYPES) === Seq(LocalResourceType.ARCHIVE.name()))
-  }
-
-}


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


[09/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala
new file mode 100644
index 0000000..7e76f40
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala
@@ -0,0 +1,235 @@
+/*
+ * 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.security
+
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{Executors, TimeUnit}
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.security.UserGroupInformation
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * The following methods are primarily meant to make sure long-running apps like Spark
+ * Streaming apps can run without interruption while accessing secured services. The
+ * scheduleLoginFromKeytab method is called on the AM to get the new credentials.
+ * This method wakes up a thread that logs into the KDC
+ * once 75% of the renewal interval of the original credentials used for the container
+ * has elapsed. It then obtains new credentials and writes them to HDFS in a
+ * pre-specified location - the prefix of which is specified in the sparkConf by
+ * spark.yarn.credentials.file (so the file(s) would be named c-timestamp1-1, c-timestamp2-2 etc.
+ * - each update goes to a new file, with a monotonically increasing suffix), also the
+ * timestamp1, timestamp2 here indicates the time of next update for CredentialUpdater.
+ * After this, the credentials are renewed once 75% of the new tokens renewal interval has elapsed.
+ *
+ * On the executor and driver (yarn client mode) side, the updateCredentialsIfRequired method is
+ * called once 80% of the validity of the original credentials has elapsed. At that time the
+ * executor finds the credentials file with the latest timestamp and checks if it has read those
+ * credentials before (by keeping track of the suffix of the last file it read). If a new file has
+ * appeared, it will read the credentials and update the currently running UGI with it. This
+ * process happens again once 80% of the validity of this has expired.
+ */
+private[yarn] class AMCredentialRenewer(
+    sparkConf: SparkConf,
+    hadoopConf: Configuration,
+    credentialManager: ConfigurableCredentialManager) extends Logging {
+
+  private var lastCredentialsFileSuffix = 0
+
+  private val credentialRenewer =
+    Executors.newSingleThreadScheduledExecutor(
+      ThreadUtils.namedThreadFactory("Credential Refresh Thread"))
+
+  private val hadoopUtil = YarnSparkHadoopUtil.get
+
+  private val credentialsFile = sparkConf.get(CREDENTIALS_FILE_PATH)
+  private val daysToKeepFiles = sparkConf.get(CREDENTIALS_FILE_MAX_RETENTION)
+  private val numFilesToKeep = sparkConf.get(CREDENTIAL_FILE_MAX_COUNT)
+  private val freshHadoopConf =
+    hadoopUtil.getConfBypassingFSCache(hadoopConf, new Path(credentialsFile).toUri.getScheme)
+
+  @volatile private var timeOfNextRenewal = sparkConf.get(CREDENTIALS_RENEWAL_TIME)
+
+  /**
+   * Schedule a login from the keytab and principal set using the --principal and --keytab
+   * arguments to spark-submit. This login happens only when the credentials of the current user
+   * are about to expire. This method reads spark.yarn.principal and spark.yarn.keytab from
+   * SparkConf to do the login. This method is a no-op in non-YARN mode.
+   *
+   */
+  private[spark] def scheduleLoginFromKeytab(): Unit = {
+    val principal = sparkConf.get(PRINCIPAL).get
+    val keytab = sparkConf.get(KEYTAB).get
+
+    /**
+     * Schedule re-login and creation of new credentials. If credentials have already expired, this
+     * method will synchronously create new ones.
+     */
+    def scheduleRenewal(runnable: Runnable): Unit = {
+      // Run now!
+      val remainingTime = timeOfNextRenewal - System.currentTimeMillis()
+      if (remainingTime <= 0) {
+        logInfo("Credentials have expired, creating new ones now.")
+        runnable.run()
+      } else {
+        logInfo(s"Scheduling login from keytab in $remainingTime millis.")
+        credentialRenewer.schedule(runnable, remainingTime, TimeUnit.MILLISECONDS)
+      }
+    }
+
+    // This thread periodically runs on the AM to update the credentials on HDFS.
+    val credentialRenewerRunnable =
+      new Runnable {
+        override def run(): Unit = {
+          try {
+            writeNewCredentialsToHDFS(principal, keytab)
+            cleanupOldFiles()
+          } catch {
+            case e: Exception =>
+              // Log the error and try to write new tokens back in an hour
+              logWarning("Failed to write out new credentials to HDFS, will try again in an " +
+                "hour! If this happens too often tasks will fail.", e)
+              credentialRenewer.schedule(this, 1, TimeUnit.HOURS)
+              return
+          }
+          scheduleRenewal(this)
+        }
+      }
+    // Schedule update of credentials. This handles the case of updating the credentials right now
+    // as well, since the renewal interval will be 0, and the thread will get scheduled
+    // immediately.
+    scheduleRenewal(credentialRenewerRunnable)
+  }
+
+  // Keeps only files that are newer than daysToKeepFiles days, and deletes everything else. At
+  // least numFilesToKeep files are kept for safety
+  private def cleanupOldFiles(): Unit = {
+    import scala.concurrent.duration._
+    try {
+      val remoteFs = FileSystem.get(freshHadoopConf)
+      val credentialsPath = new Path(credentialsFile)
+      val thresholdTime = System.currentTimeMillis() - (daysToKeepFiles.days).toMillis
+      hadoopUtil.listFilesSorted(
+        remoteFs, credentialsPath.getParent,
+        credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
+        .dropRight(numFilesToKeep)
+        .takeWhile(_.getModificationTime < thresholdTime)
+        .foreach(x => remoteFs.delete(x.getPath, true))
+    } catch {
+      // Such errors are not fatal, so don't throw. Make sure they are logged though
+      case e: Exception =>
+        logWarning("Error while attempting to cleanup old credentials. If you are seeing many " +
+          "such warnings there may be an issue with your HDFS cluster.", e)
+    }
+  }
+
+  private def writeNewCredentialsToHDFS(principal: String, keytab: String): Unit = {
+    // Keytab is copied by YARN to the working directory of the AM, so full path is
+    // not needed.
+
+    // HACK:
+    // HDFS will not issue new delegation tokens, if the Credentials object
+    // passed in already has tokens for that FS even if the tokens are expired (it really only
+    // checks if there are tokens for the service, and not if they are valid). So the only real
+    // way to get new tokens is to make sure a different Credentials object is used each time to
+    // get new tokens and then the new tokens are copied over the current user's Credentials.
+    // So:
+    // - we login as a different user and get the UGI
+    // - use that UGI to get the tokens (see doAs block below)
+    // - copy the tokens over to the current user's credentials (this will overwrite the tokens
+    // in the current user's Credentials object for this FS).
+    // The login to KDC happens each time new tokens are required, but this is rare enough to not
+    // have to worry about (like once every day or so). This makes this code clearer than having
+    // to login and then relogin every time (the HDFS API may not relogin since we don't use this
+    // UGI directly for HDFS communication.
+    logInfo(s"Attempting to login to KDC using principal: $principal")
+    val keytabLoggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab)
+    logInfo("Successfully logged into KDC.")
+    val tempCreds = keytabLoggedInUGI.getCredentials
+    val credentialsPath = new Path(credentialsFile)
+    val dst = credentialsPath.getParent
+    var nearestNextRenewalTime = Long.MaxValue
+    keytabLoggedInUGI.doAs(new PrivilegedExceptionAction[Void] {
+      // Get a copy of the credentials
+      override def run(): Void = {
+        nearestNextRenewalTime = credentialManager.obtainCredentials(freshHadoopConf, tempCreds)
+        null
+      }
+    })
+
+    val currTime = System.currentTimeMillis()
+    val timeOfNextUpdate = if (nearestNextRenewalTime <= currTime) {
+      // If next renewal time is earlier than current time, we set next renewal time to current
+      // time, this will trigger next renewal immediately. Also set next update time to current
+      // time. There still has a gap between token renewal and update will potentially introduce
+      // issue.
+      logWarning(s"Next credential renewal time ($nearestNextRenewalTime) is earlier than " +
+        s"current time ($currTime), which is unexpected, please check your credential renewal " +
+        "related configurations in the target services.")
+      timeOfNextRenewal = currTime
+      currTime
+    } else {
+      // Next valid renewal time is about 75% of credential renewal time, and update time is
+      // slightly later than valid renewal time (80% of renewal time).
+      timeOfNextRenewal = ((nearestNextRenewalTime - currTime) * 0.75 + currTime).toLong
+      ((nearestNextRenewalTime - currTime) * 0.8 + currTime).toLong
+    }
+
+    // Add the temp credentials back to the original ones.
+    UserGroupInformation.getCurrentUser.addCredentials(tempCreds)
+    val remoteFs = FileSystem.get(freshHadoopConf)
+    // If lastCredentialsFileSuffix is 0, then the AM is either started or restarted. If the AM
+    // was restarted, then the lastCredentialsFileSuffix might be > 0, so find the newest file
+    // and update the lastCredentialsFileSuffix.
+    if (lastCredentialsFileSuffix == 0) {
+      hadoopUtil.listFilesSorted(
+        remoteFs, credentialsPath.getParent,
+        credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
+        .lastOption.foreach { status =>
+        lastCredentialsFileSuffix = hadoopUtil.getSuffixForCredentialsPath(status.getPath)
+      }
+    }
+    val nextSuffix = lastCredentialsFileSuffix + 1
+
+    val tokenPathStr =
+      credentialsFile + SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM +
+        timeOfNextUpdate.toLong.toString + SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM +
+          nextSuffix
+    val tokenPath = new Path(tokenPathStr)
+    val tempTokenPath = new Path(tokenPathStr + SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
+
+    logInfo("Writing out delegation tokens to " + tempTokenPath.toString)
+    val credentials = UserGroupInformation.getCurrentUser.getCredentials
+    credentials.writeTokenStorageFile(tempTokenPath, freshHadoopConf)
+    logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr")
+    remoteFs.rename(tempTokenPath, tokenPath)
+    logInfo("Delegation token file rename complete.")
+    lastCredentialsFileSuffix = nextSuffix
+  }
+
+  def stop(): Unit = {
+    credentialRenewer.shutdown()
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala
new file mode 100644
index 0000000..c4c07b4
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala
@@ -0,0 +1,105 @@
+/*
+ * 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.security
+
+import java.util.ServiceLoader
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.Credentials
+
+import org.apache.spark.SparkConf
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+/**
+ * A ConfigurableCredentialManager to manage all the registered credential providers and offer
+ * APIs for other modules to obtain credentials as well as renewal time. By default
+ * [[HDFSCredentialProvider]], [[HiveCredentialProvider]] and [[HBaseCredentialProvider]] will
+ * be loaded in if not explicitly disabled, any plugged-in credential provider wants to be
+ * managed by ConfigurableCredentialManager needs to implement [[ServiceCredentialProvider]]
+ * interface and put into resources/META-INF/services to be loaded by ServiceLoader.
+ *
+ * Also each credential provider is controlled by
+ * spark.yarn.security.credentials.{service}.enabled, it will not be loaded in if set to false.
+ */
+private[yarn] final class ConfigurableCredentialManager(
+    sparkConf: SparkConf, hadoopConf: Configuration) extends Logging {
+  private val deprecatedProviderEnabledConfig = "spark.yarn.security.tokens.%s.enabled"
+  private val providerEnabledConfig = "spark.yarn.security.credentials.%s.enabled"
+
+  // Maintain all the registered credential providers
+  private val credentialProviders = {
+    val providers = ServiceLoader.load(classOf[ServiceCredentialProvider],
+      Utils.getContextOrSparkClassLoader).asScala
+
+    // Filter out credentials in which spark.yarn.security.credentials.{service}.enabled is false.
+    providers.filter { p =>
+      sparkConf.getOption(providerEnabledConfig.format(p.serviceName))
+        .orElse {
+          sparkConf.getOption(deprecatedProviderEnabledConfig.format(p.serviceName)).map { c =>
+            logWarning(s"${deprecatedProviderEnabledConfig.format(p.serviceName)} is deprecated, " +
+              s"using ${providerEnabledConfig.format(p.serviceName)} instead")
+            c
+          }
+        }.map(_.toBoolean).getOrElse(true)
+    }.map { p => (p.serviceName, p) }.toMap
+  }
+
+  /**
+   * Get credential provider for the specified service.
+   */
+  def getServiceCredentialProvider(service: String): Option[ServiceCredentialProvider] = {
+    credentialProviders.get(service)
+  }
+
+  /**
+   * Obtain credentials from all the registered providers.
+   * @return nearest time of next renewal, Long.MaxValue if all the credentials aren't renewable,
+   *         otherwise the nearest renewal time of any credentials will be returned.
+   */
+  def obtainCredentials(hadoopConf: Configuration, creds: Credentials): Long = {
+    credentialProviders.values.flatMap { provider =>
+      if (provider.credentialsRequired(hadoopConf)) {
+        provider.obtainCredentials(hadoopConf, sparkConf, creds)
+      } else {
+        logDebug(s"Service ${provider.serviceName} does not require a token." +
+          s" Check your configuration to see if security is disabled or not.")
+        None
+      }
+    }.foldLeft(Long.MaxValue)(math.min)
+  }
+
+  /**
+   * Create an [[AMCredentialRenewer]] instance, caller should be responsible to stop this
+   * instance when it is not used. AM will use it to renew credentials periodically.
+   */
+  def credentialRenewer(): AMCredentialRenewer = {
+    new AMCredentialRenewer(sparkConf, hadoopConf, this)
+  }
+
+  /**
+   * Create an [[CredentialUpdater]] instance, caller should be resposible to stop this intance
+   * when it is not used. Executors and driver (client mode) will use it to update credentials.
+   * periodically.
+   */
+  def credentialUpdater(): CredentialUpdater = {
+    new CredentialUpdater(sparkConf, hadoopConf, this)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala
new file mode 100644
index 0000000..5df4fbd
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala
@@ -0,0 +1,130 @@
+/*
+ * 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.security
+
+import java.util.concurrent.{Executors, TimeUnit}
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.{ThreadUtils, Utils}
+
+private[spark] class CredentialUpdater(
+    sparkConf: SparkConf,
+    hadoopConf: Configuration,
+    credentialManager: ConfigurableCredentialManager) extends Logging {
+
+  @volatile private var lastCredentialsFileSuffix = 0
+
+  private val credentialsFile = sparkConf.get(CREDENTIALS_FILE_PATH)
+  private val freshHadoopConf =
+    SparkHadoopUtil.get.getConfBypassingFSCache(
+      hadoopConf, new Path(credentialsFile).toUri.getScheme)
+
+  private val credentialUpdater =
+    Executors.newSingleThreadScheduledExecutor(
+      ThreadUtils.namedThreadFactory("Credential Refresh Thread"))
+
+  // This thread wakes up and picks up new credentials from HDFS, if any.
+  private val credentialUpdaterRunnable =
+    new Runnable {
+      override def run(): Unit = Utils.logUncaughtExceptions(updateCredentialsIfRequired())
+    }
+
+  /** Start the credential updater task */
+  def start(): Unit = {
+    val startTime = sparkConf.get(CREDENTIALS_RENEWAL_TIME)
+    val remainingTime = startTime - System.currentTimeMillis()
+    if (remainingTime <= 0) {
+      credentialUpdater.schedule(credentialUpdaterRunnable, 1, TimeUnit.MINUTES)
+    } else {
+      logInfo(s"Scheduling credentials refresh from HDFS in $remainingTime millis.")
+      credentialUpdater.schedule(credentialUpdaterRunnable, remainingTime, TimeUnit.MILLISECONDS)
+    }
+  }
+
+  private def updateCredentialsIfRequired(): Unit = {
+    val timeToNextUpdate = try {
+      val credentialsFilePath = new Path(credentialsFile)
+      val remoteFs = FileSystem.get(freshHadoopConf)
+      SparkHadoopUtil.get.listFilesSorted(
+        remoteFs, credentialsFilePath.getParent,
+        credentialsFilePath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
+        .lastOption.map { credentialsStatus =>
+          val suffix = SparkHadoopUtil.get.getSuffixForCredentialsPath(credentialsStatus.getPath)
+          if (suffix > lastCredentialsFileSuffix) {
+            logInfo("Reading new credentials from " + credentialsStatus.getPath)
+            val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath)
+            lastCredentialsFileSuffix = suffix
+            UserGroupInformation.getCurrentUser.addCredentials(newCredentials)
+            logInfo("Credentials updated from credentials file.")
+
+            val remainingTime = getTimeOfNextUpdateFromFileName(credentialsStatus.getPath)
+              - System.currentTimeMillis()
+            if (remainingTime <= 0) TimeUnit.MINUTES.toMillis(1) else remainingTime
+          } else {
+            // If current credential file is older than expected, sleep 1 hour and check again.
+            TimeUnit.HOURS.toMillis(1)
+          }
+      }.getOrElse {
+        // Wait for 1 minute to check again if there's no credential file currently
+        TimeUnit.MINUTES.toMillis(1)
+      }
+    } catch {
+      // Since the file may get deleted while we are reading it, catch the Exception and come
+      // back in an hour to try again
+      case NonFatal(e) =>
+        logWarning("Error while trying to update credentials, will try again in 1 hour", e)
+        TimeUnit.HOURS.toMillis(1)
+    }
+
+    credentialUpdater.schedule(
+      credentialUpdaterRunnable, timeToNextUpdate, TimeUnit.MILLISECONDS)
+  }
+
+  private def getCredentialsFromHDFSFile(remoteFs: FileSystem, tokenPath: Path): Credentials = {
+    val stream = remoteFs.open(tokenPath)
+    try {
+      val newCredentials = new Credentials()
+      newCredentials.readTokenStorageStream(stream)
+      newCredentials
+    } finally {
+      stream.close()
+    }
+  }
+
+  private def getTimeOfNextUpdateFromFileName(credentialsPath: Path): Long = {
+    val name = credentialsPath.getName
+    val index = name.lastIndexOf(SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM)
+    val slice = name.substring(0, index)
+    val last2index = slice.lastIndexOf(SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM)
+    name.substring(last2index + 1, index).toLong
+  }
+
+  def stop(): Unit = {
+    credentialUpdater.shutdown()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala
new file mode 100644
index 0000000..5571df0
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.yarn.security
+
+import scala.reflect.runtime.universe
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.Credentials
+import org.apache.hadoop.security.token.{Token, TokenIdentifier}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.internal.Logging
+
+private[security] class HBaseCredentialProvider extends ServiceCredentialProvider with Logging {
+
+  override def serviceName: String = "hbase"
+
+  override def obtainCredentials(
+      hadoopConf: Configuration,
+      sparkConf: SparkConf,
+      creds: Credentials): Option[Long] = {
+    try {
+      val mirror = universe.runtimeMirror(getClass.getClassLoader)
+      val obtainToken = mirror.classLoader.
+        loadClass("org.apache.hadoop.hbase.security.token.TokenUtil").
+        getMethod("obtainToken", classOf[Configuration])
+
+      logDebug("Attempting to fetch HBase security token.")
+      val token = obtainToken.invoke(null, hbaseConf(hadoopConf))
+        .asInstanceOf[Token[_ <: TokenIdentifier]]
+      logInfo(s"Get token from HBase: ${token.toString}")
+      creds.addToken(token.getService, token)
+    } catch {
+      case NonFatal(e) =>
+        logDebug(s"Failed to get token from service $serviceName", e)
+    }
+
+    None
+  }
+
+  override def credentialsRequired(hadoopConf: Configuration): Boolean = {
+    hbaseConf(hadoopConf).get("hbase.security.authentication") == "kerberos"
+  }
+
+  private def hbaseConf(conf: Configuration): Configuration = {
+    try {
+      val mirror = universe.runtimeMirror(getClass.getClassLoader)
+      val confCreate = mirror.classLoader.
+        loadClass("org.apache.hadoop.hbase.HBaseConfiguration").
+        getMethod("create", classOf[Configuration])
+      confCreate.invoke(null, conf).asInstanceOf[Configuration]
+    } catch {
+      case NonFatal(e) =>
+        logDebug("Fail to invoke HBaseConfiguration", e)
+        conf
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProvider.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProvider.scala
new file mode 100644
index 0000000..8d06d73
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProvider.scala
@@ -0,0 +1,110 @@
+/*
+ * 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.security
+
+import java.io.{ByteArrayInputStream, DataInputStream}
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier
+import org.apache.hadoop.mapred.Master
+import org.apache.hadoop.security.Credentials
+
+import org.apache.spark.{SparkConf, SparkException}
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+
+private[security] class HDFSCredentialProvider extends ServiceCredentialProvider with Logging {
+  // Token renewal interval, this value will be set in the first call,
+  // if None means no token renewer specified, so cannot get token renewal interval.
+  private var tokenRenewalInterval: Option[Long] = null
+
+  override val serviceName: String = "hdfs"
+
+  override def obtainCredentials(
+      hadoopConf: Configuration,
+      sparkConf: SparkConf,
+      creds: Credentials): Option[Long] = {
+    // NameNode to access, used to get tokens from different FileSystems
+    nnsToAccess(hadoopConf, sparkConf).foreach { dst =>
+      val dstFs = dst.getFileSystem(hadoopConf)
+      logInfo("getting token for namenode: " + dst)
+      dstFs.addDelegationTokens(getTokenRenewer(hadoopConf), creds)
+    }
+
+    // Get the token renewal interval if it is not set. It will only be called once.
+    if (tokenRenewalInterval == null) {
+      tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf)
+    }
+
+    // Get the time of next renewal.
+    tokenRenewalInterval.map { interval =>
+      creds.getAllTokens.asScala
+        .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND)
+        .map { t =>
+          val identifier = new DelegationTokenIdentifier()
+          identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier)))
+          identifier.getIssueDate + interval
+      }.foldLeft(0L)(math.max)
+    }
+  }
+
+  private def getTokenRenewalInterval(
+      hadoopConf: Configuration, sparkConf: SparkConf): Option[Long] = {
+    // We cannot use the tokens generated with renewer yarn. Trying to renew
+    // those will fail with an access control issue. So create new tokens with the logged in
+    // user as renewer.
+    sparkConf.get(PRINCIPAL).map { renewer =>
+      val creds = new Credentials()
+      nnsToAccess(hadoopConf, sparkConf).foreach { dst =>
+        val dstFs = dst.getFileSystem(hadoopConf)
+        dstFs.addDelegationTokens(renewer, creds)
+      }
+      val t = creds.getAllTokens.asScala
+        .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND)
+        .head
+      val newExpiration = t.renew(hadoopConf)
+      val identifier = new DelegationTokenIdentifier()
+      identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier)))
+      val interval = newExpiration - identifier.getIssueDate
+      logInfo(s"Renewal Interval is $interval")
+      interval
+    }
+  }
+
+  private def getTokenRenewer(conf: Configuration): String = {
+    val delegTokenRenewer = Master.getMasterPrincipal(conf)
+    logDebug("delegation token renewer is: " + delegTokenRenewer)
+    if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
+      val errorMessage = "Can't get Master Kerberos principal for use as renewer"
+      logError(errorMessage)
+      throw new SparkException(errorMessage)
+    }
+
+    delegTokenRenewer
+  }
+
+  private def nnsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = {
+    sparkConf.get(NAMENODES_TO_ACCESS).map(new Path(_)).toSet +
+      sparkConf.get(STAGING_DIR).map(new Path(_))
+        .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala
new file mode 100644
index 0000000..16d8fc3
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala
@@ -0,0 +1,129 @@
+/*
+ * 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.security
+
+import java.lang.reflect.UndeclaredThrowableException
+import java.security.PrivilegedExceptionAction
+
+import scala.reflect.runtime.universe
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+import org.apache.hadoop.security.token.Token
+
+import org.apache.spark.SparkConf
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+private[security] class HiveCredentialProvider extends ServiceCredentialProvider with Logging {
+
+  override def serviceName: String = "hive"
+
+  private def hiveConf(hadoopConf: Configuration): Configuration = {
+    try {
+      val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
+      // the hive configuration class is a subclass of Hadoop Configuration, so can be cast down
+      // to a Configuration and used without reflection
+      val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf")
+      // using the (Configuration, Class) constructor allows the current configuration to be
+      // included in the hive config.
+      val ctor = hiveConfClass.getDeclaredConstructor(classOf[Configuration],
+        classOf[Object].getClass)
+      ctor.newInstance(hadoopConf, hiveConfClass).asInstanceOf[Configuration]
+    } catch {
+      case NonFatal(e) =>
+        logDebug("Fail to create Hive Configuration", e)
+        hadoopConf
+    }
+  }
+
+  override def credentialsRequired(hadoopConf: Configuration): Boolean = {
+    UserGroupInformation.isSecurityEnabled &&
+      hiveConf(hadoopConf).getTrimmed("hive.metastore.uris", "").nonEmpty
+  }
+
+  override def obtainCredentials(
+      hadoopConf: Configuration,
+      sparkConf: SparkConf,
+      creds: Credentials): Option[Long] = {
+    val conf = hiveConf(hadoopConf)
+
+    val principalKey = "hive.metastore.kerberos.principal"
+    val principal = conf.getTrimmed(principalKey, "")
+    require(principal.nonEmpty, s"Hive principal $principalKey undefined")
+    val metastoreUri = conf.getTrimmed("hive.metastore.uris", "")
+    require(metastoreUri.nonEmpty, "Hive metastore uri undefined")
+
+    val currentUser = UserGroupInformation.getCurrentUser()
+    logDebug(s"Getting Hive delegation token for ${currentUser.getUserName()} against " +
+      s"$principal at $metastoreUri")
+
+    val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
+    val hiveClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.ql.metadata.Hive")
+    val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf")
+    val closeCurrent = hiveClass.getMethod("closeCurrent")
+
+    try {
+      // get all the instance methods before invoking any
+      val getDelegationToken = hiveClass.getMethod("getDelegationToken",
+        classOf[String], classOf[String])
+      val getHive = hiveClass.getMethod("get", hiveConfClass)
+
+      doAsRealUser {
+        val hive = getHive.invoke(null, conf)
+        val tokenStr = getDelegationToken.invoke(hive, currentUser.getUserName(), principal)
+          .asInstanceOf[String]
+        val hive2Token = new Token[DelegationTokenIdentifier]()
+        hive2Token.decodeFromUrlString(tokenStr)
+        logInfo(s"Get Token from hive metastore: ${hive2Token.toString}")
+        creds.addToken(new Text("hive.server2.delegation.token"), hive2Token)
+      }
+    } catch {
+      case NonFatal(e) =>
+        logDebug(s"Fail to get token from service $serviceName", e)
+    } finally {
+      Utils.tryLogNonFatalError {
+        closeCurrent.invoke(null)
+      }
+    }
+
+    None
+  }
+
+  /**
+   * Run some code as the real logged in user (which may differ from the current user, for
+   * example, when using proxying).
+   */
+  private def doAsRealUser[T](fn: => T): T = {
+    val currentUser = UserGroupInformation.getCurrentUser()
+    val realUser = Option(currentUser.getRealUser()).getOrElse(currentUser)
+
+   // For some reason the Scala-generated anonymous class ends up causing an
+   // UndeclaredThrowableException, even if you annotate the method with @throws.
+   try {
+      realUser.doAs(new PrivilegedExceptionAction[T]() {
+        override def run(): T = fn
+      })
+    } catch {
+      case e: UndeclaredThrowableException => throw Option(e.getCause()).getOrElse(e)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala
new file mode 100644
index 0000000..4e3fcce
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala
@@ -0,0 +1,57 @@
+/*
+ * 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.security
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+
+/**
+ * A credential provider for a service. User must implement this if they need to access a
+ * secure service from Spark.
+ */
+trait ServiceCredentialProvider {
+
+  /**
+   * Name of the service to provide credentials. This name should unique, Spark internally will
+   * use this name to differentiate credential provider.
+   */
+  def serviceName: String
+
+  /**
+   * To decide whether credential is required for this service. By default it based on whether
+   * Hadoop security is enabled.
+   */
+  def credentialsRequired(hadoopConf: Configuration): Boolean = {
+    UserGroupInformation.isSecurityEnabled
+  }
+
+  /**
+   * Obtain credentials for this service and get the time of the next renewal.
+   * @param hadoopConf Configuration of current Hadoop Compatible system.
+   * @param sparkConf Spark configuration.
+   * @param creds Credentials to add tokens and security keys to.
+   * @return If this Credential is renewable and can be renewed, return the time of the next
+   *         renewal, otherwise None should be returned.
+   */
+  def obtainCredentials(
+      hadoopConf: Configuration,
+      sparkConf: SparkConf,
+      creds: Credentials): Option[Long]
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
new file mode 100644
index 0000000..6c3556a
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.launcher
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+import scala.util.Properties
+
+/**
+ * Exposes methods from the launcher library that are used by the YARN backend.
+ */
+private[spark] object YarnCommandBuilderUtils {
+
+  def quoteForBatchScript(arg: String): String = {
+    CommandBuilderUtils.quoteForBatchScript(arg)
+  }
+
+  def findJarsDir(sparkHome: String): String = {
+    val scalaVer = Properties.versionNumberString
+      .split("\\.")
+      .take(2)
+      .mkString(".")
+    CommandBuilderUtils.findJarsDir(sparkHome, scalaVer, true)
+  }
+
+  /**
+   * Adds the perm gen configuration to the list of java options if needed and not yet added.
+   *
+   * Note that this method adds the option based on the local JVM version; if the node where
+   * the container is running has a different Java version, there's a risk that the option will
+   * not be added (e.g. if the AM is running Java 8 but the container's node is set up to use
+   * Java 7).
+   */
+  def addPermGenSizeOpt(args: ListBuffer[String]): Unit = {
+    CommandBuilderUtils.addPermGenSizeOpt(args.asJava)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala
new file mode 100644
index 0000000..4ed2852
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala
@@ -0,0 +1,143 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicBoolean
+
+import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
+
+import org.apache.spark.SparkContext
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+/**
+ * An extension service that can be loaded into a Spark YARN scheduler.
+ * A Service that can be started and stopped.
+ *
+ * 1. For implementations to be loadable by `SchedulerExtensionServices`,
+ * they must provide an empty constructor.
+ * 2. The `stop()` operation MUST be idempotent, and succeed even if `start()` was
+ * never invoked.
+ */
+trait SchedulerExtensionService {
+
+  /**
+   * Start the extension service. This should be a no-op if
+   * called more than once.
+   * @param binding binding to the spark application and YARN
+   */
+  def start(binding: SchedulerExtensionServiceBinding): Unit
+
+  /**
+   * Stop the service
+   * The `stop()` operation MUST be idempotent, and succeed even if `start()` was
+   * never invoked.
+   */
+  def stop(): Unit
+}
+
+/**
+ * Binding information for a [[SchedulerExtensionService]].
+ *
+ * The attempt ID will be set if the service is started within a YARN application master;
+ * there is then a different attempt ID for every time that AM is restarted.
+ * When the service binding is instantiated in client mode, there's no attempt ID, as it lacks
+ * this information.
+ * @param sparkContext current spark context
+ * @param applicationId YARN application ID
+ * @param attemptId YARN attemptID. This will always be unset in client mode, and always set in
+ *                  cluster mode.
+ */
+case class SchedulerExtensionServiceBinding(
+    sparkContext: SparkContext,
+    applicationId: ApplicationId,
+    attemptId: Option[ApplicationAttemptId] = None)
+
+/**
+ * Container for [[SchedulerExtensionService]] instances.
+ *
+ * Loads Extension Services from the configuration property
+ * `"spark.yarn.services"`, instantiates and starts them.
+ * When stopped, it stops all child entries.
+ *
+ * The order in which child extension services are started and stopped
+ * is undefined.
+ */
+private[spark] class SchedulerExtensionServices extends SchedulerExtensionService
+    with Logging {
+  private var serviceOption: Option[String] = None
+  private var services: List[SchedulerExtensionService] = Nil
+  private val started = new AtomicBoolean(false)
+  private var binding: SchedulerExtensionServiceBinding = _
+
+  /**
+   * Binding operation will load the named services and call bind on them too; the
+   * entire set of services are then ready for `init()` and `start()` calls.
+   *
+   * @param binding binding to the spark application and YARN
+   */
+  def start(binding: SchedulerExtensionServiceBinding): Unit = {
+    if (started.getAndSet(true)) {
+      logWarning("Ignoring re-entrant start operation")
+      return
+    }
+    require(binding.sparkContext != null, "Null context parameter")
+    require(binding.applicationId != null, "Null appId parameter")
+    this.binding = binding
+    val sparkContext = binding.sparkContext
+    val appId = binding.applicationId
+    val attemptId = binding.attemptId
+    logInfo(s"Starting Yarn extension services with app $appId and attemptId $attemptId")
+
+    services = sparkContext.conf.get(SCHEDULER_SERVICES).map { sClass =>
+      val instance = Utils.classForName(sClass)
+        .newInstance()
+        .asInstanceOf[SchedulerExtensionService]
+      // bind this service
+      instance.start(binding)
+      logInfo(s"Service $sClass started")
+      instance
+    }.toList
+  }
+
+  /**
+   * Get the list of services.
+   *
+   * @return a list of services; Nil until the service is started
+   */
+  def getServices: List[SchedulerExtensionService] = services
+
+  /**
+   * Stop the services; idempotent.
+   *
+   */
+  override def stop(): Unit = {
+    if (started.getAndSet(false)) {
+      logInfo(s"Stopping $this")
+      services.foreach { s =>
+        Utils.tryLogNonFatalError(s.stop())
+      }
+    }
+  }
+
+  override def toString(): String = s"""SchedulerExtensionServices
+    |(serviceOption=$serviceOption,
+    | services=$services,
+    | started=$started)""".stripMargin
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
new file mode 100644
index 0000000..60da356
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
@@ -0,0 +1,157 @@
+/*
+ * 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.mutable.ArrayBuffer
+
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
+
+import org.apache.spark.{SparkContext, SparkException}
+import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnSparkHadoopUtil}
+import org.apache.spark.internal.Logging
+import org.apache.spark.launcher.SparkAppHandle
+import org.apache.spark.scheduler.TaskSchedulerImpl
+
+private[spark] class YarnClientSchedulerBackend(
+    scheduler: TaskSchedulerImpl,
+    sc: SparkContext)
+  extends YarnSchedulerBackend(scheduler, sc)
+  with Logging {
+
+  private var client: Client = null
+  private var monitorThread: MonitorThread = null
+
+  /**
+   * Create a Yarn client to submit an application to the ResourceManager.
+   * This waits until the application is running.
+   */
+  override def start() {
+    val driverHost = conf.get("spark.driver.host")
+    val driverPort = conf.get("spark.driver.port")
+    val hostport = driverHost + ":" + driverPort
+    sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.webUrl) }
+
+    val argsArrayBuf = new ArrayBuffer[String]()
+    argsArrayBuf += ("--arg", hostport)
+
+    logDebug("ClientArguments called with: " + argsArrayBuf.mkString(" "))
+    val args = new ClientArguments(argsArrayBuf.toArray)
+    totalExpectedExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(conf)
+    client = new Client(args, conf)
+    bindToYarn(client.submitApplication(), None)
+
+    // SPARK-8687: Ensure all necessary properties have already been set before
+    // we initialize our driver scheduler backend, which serves these properties
+    // to the executors
+    super.start()
+    waitForApplication()
+
+    // SPARK-8851: In yarn-client mode, the AM still does the credentials refresh. The driver
+    // reads the credentials from HDFS, just like the executors and updates its own credentials
+    // cache.
+    if (conf.contains("spark.yarn.credentials.file")) {
+      YarnSparkHadoopUtil.get.startCredentialUpdater(conf)
+    }
+    monitorThread = asyncMonitorApplication()
+    monitorThread.start()
+  }
+
+  /**
+   * Report the state of the application until it is running.
+   * If the application has finished, failed or been killed in the process, throw an exception.
+   * This assumes both `client` and `appId` have already been set.
+   */
+  private def waitForApplication(): Unit = {
+    assert(client != null && appId.isDefined, "Application has not been submitted yet!")
+    val (state, _) = client.monitorApplication(appId.get, returnOnRunning = true) // blocking
+    if (state == YarnApplicationState.FINISHED ||
+      state == YarnApplicationState.FAILED ||
+      state == YarnApplicationState.KILLED) {
+      throw new SparkException("Yarn application has already ended! " +
+        "It might have been killed or unable to launch application master.")
+    }
+    if (state == YarnApplicationState.RUNNING) {
+      logInfo(s"Application ${appId.get} has started running.")
+    }
+  }
+
+  /**
+   * We create this class for SPARK-9519. Basically when we interrupt the monitor thread it's
+   * because the SparkContext is being shut down(sc.stop() called by user code), but if
+   * monitorApplication return, it means the Yarn application finished before sc.stop() was called,
+   * which means we should call sc.stop() here, and we don't allow the monitor to be interrupted
+   * before SparkContext stops successfully.
+   */
+  private class MonitorThread extends Thread {
+    private var allowInterrupt = true
+
+    override def run() {
+      try {
+        val (state, _) = client.monitorApplication(appId.get, logApplicationReport = false)
+        logError(s"Yarn application has already exited with state $state!")
+        allowInterrupt = false
+        sc.stop()
+      } catch {
+        case e: InterruptedException => logInfo("Interrupting monitor thread")
+      }
+    }
+
+    def stopMonitor(): Unit = {
+      if (allowInterrupt) {
+        this.interrupt()
+      }
+    }
+  }
+
+  /**
+   * Monitor the application state in a separate thread.
+   * If the application has exited for any reason, stop the SparkContext.
+   * This assumes both `client` and `appId` have already been set.
+   */
+  private def asyncMonitorApplication(): MonitorThread = {
+    assert(client != null && appId.isDefined, "Application has not been submitted yet!")
+    val t = new MonitorThread
+    t.setName("Yarn application state monitor")
+    t.setDaemon(true)
+    t
+  }
+
+  /**
+   * Stop the scheduler. This assumes `start()` has already been called.
+   */
+  override def stop() {
+    assert(client != null, "Attempted to stop this scheduler before starting it!")
+    if (monitorThread != null) {
+      monitorThread.stopMonitor()
+    }
+
+    // Report a final state to the launcher if one is connected. This is needed since in client
+    // mode this backend doesn't let the app monitor loop run to completion, so it does not report
+    // the final state itself.
+    //
+    // Note: there's not enough information at this point to provide a better final state,
+    // so assume the application was successful.
+    client.reportLauncherState(SparkAppHandle.State.FINISHED)
+
+    super.stop()
+    YarnSparkHadoopUtil.get.stopCredentialUpdater()
+    client.stop()
+    logInfo("Stopped")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala
new file mode 100644
index 0000000..64cd1bd
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler.cluster
+
+import org.apache.spark.{SparkContext, SparkException}
+import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
+
+/**
+ * Cluster Manager for creation of Yarn scheduler and backend
+ */
+private[spark] class YarnClusterManager extends ExternalClusterManager {
+
+  override def canCreate(masterURL: String): Boolean = {
+    masterURL == "yarn"
+  }
+
+  override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = {
+    sc.deployMode match {
+      case "cluster" => new YarnClusterScheduler(sc)
+      case "client" => new YarnScheduler(sc)
+      case _ => throw new SparkException(s"Unknown deploy mode '${sc.deployMode}' for Yarn")
+    }
+  }
+
+  override def createSchedulerBackend(sc: SparkContext,
+      masterURL: String,
+      scheduler: TaskScheduler): SchedulerBackend = {
+    sc.deployMode match {
+      case "cluster" =>
+        new YarnClusterSchedulerBackend(scheduler.asInstanceOf[TaskSchedulerImpl], sc)
+      case "client" =>
+        new YarnClientSchedulerBackend(scheduler.asInstanceOf[TaskSchedulerImpl], sc)
+      case  _ =>
+        throw new SparkException(s"Unknown deploy mode '${sc.deployMode}' for Yarn")
+    }
+  }
+
+  override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = {
+    scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
new file mode 100644
index 0000000..96c9151
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler.cluster
+
+import org.apache.spark._
+import org.apache.spark.deploy.yarn.ApplicationMaster
+
+/**
+ * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of
+ * ApplicationMaster, etc is done
+ */
+private[spark] class YarnClusterScheduler(sc: SparkContext) extends YarnScheduler(sc) {
+
+  logInfo("Created YarnClusterScheduler")
+
+  override def postStartHook() {
+    ApplicationMaster.sparkContextInitialized(sc)
+    super.postStartHook()
+    logInfo("YarnClusterScheduler.postStartHook done")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
new file mode 100644
index 0000000..4f3d5eb
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler.cluster
+
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+
+import org.apache.spark.SparkContext
+import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnSparkHadoopUtil}
+import org.apache.spark.scheduler.TaskSchedulerImpl
+import org.apache.spark.util.Utils
+
+private[spark] class YarnClusterSchedulerBackend(
+    scheduler: TaskSchedulerImpl,
+    sc: SparkContext)
+  extends YarnSchedulerBackend(scheduler, sc) {
+
+  override def start() {
+    val attemptId = ApplicationMaster.getAttemptId
+    bindToYarn(attemptId.getApplicationId(), Some(attemptId))
+    super.start()
+    totalExpectedExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sc.conf)
+  }
+
+  override def getDriverLogUrls: Option[Map[String, String]] = {
+    var driverLogs: Option[Map[String, String]] = None
+    try {
+      val yarnConf = new YarnConfiguration(sc.hadoopConfiguration)
+      val containerId = YarnSparkHadoopUtil.get.getContainerId
+
+      val httpAddress = System.getenv(Environment.NM_HOST.name()) +
+        ":" + System.getenv(Environment.NM_HTTP_PORT.name())
+      // lookup appropriate http scheme for container log urls
+      val yarnHttpPolicy = yarnConf.get(
+        YarnConfiguration.YARN_HTTP_POLICY_KEY,
+        YarnConfiguration.YARN_HTTP_POLICY_DEFAULT
+      )
+      val user = Utils.getCurrentUserName()
+      val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://"
+      val baseUrl = s"$httpScheme$httpAddress/node/containerlogs/$containerId/$user"
+      logDebug(s"Base URL for logs: $baseUrl")
+      driverLogs = Some(Map(
+        "stdout" -> s"$baseUrl/stdout?start=-4096",
+        "stderr" -> s"$baseUrl/stderr?start=-4096"))
+    } catch {
+      case e: Exception =>
+        logInfo("Error while building AM log links, so AM" +
+          " logs link will not appear in application UI", e)
+    }
+    driverLogs
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..0293821
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler.cluster
+
+import org.apache.hadoop.yarn.util.RackResolver
+import org.apache.log4j.{Level, Logger}
+
+import org.apache.spark._
+import org.apache.spark.scheduler.TaskSchedulerImpl
+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)
+  }
+
+  // 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)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
new file mode 100644
index 0000000..2f9ea19
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
@@ -0,0 +1,315 @@
+/*
+ * 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.concurrent.{ExecutionContext, Future}
+import scala.util.{Failure, Success}
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
+
+import org.apache.spark.SparkContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rpc._
+import org.apache.spark.scheduler._
+import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
+import org.apache.spark.ui.JettyUtils
+import org.apache.spark.util.{RpcUtils, ThreadUtils}
+
+/**
+ * Abstract Yarn scheduler backend that contains common logic
+ * between the client and cluster Yarn scheduler backends.
+ */
+private[spark] abstract class YarnSchedulerBackend(
+    scheduler: TaskSchedulerImpl,
+    sc: SparkContext)
+  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) {
+
+  override val minRegisteredRatio =
+    if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+      0.8
+    } else {
+      super.minRegisteredRatio
+    }
+
+  protected var totalExpectedExecutors = 0
+
+  private val yarnSchedulerEndpoint = new YarnSchedulerEndpoint(rpcEnv)
+
+  private val yarnSchedulerEndpointRef = rpcEnv.setupEndpoint(
+    YarnSchedulerBackend.ENDPOINT_NAME, yarnSchedulerEndpoint)
+
+  private implicit val askTimeout = RpcUtils.askRpcTimeout(sc.conf)
+
+  /** Application ID. */
+  protected var appId: Option[ApplicationId] = None
+
+  /** Attempt ID. This is unset for client-mode schedulers */
+  private var attemptId: Option[ApplicationAttemptId] = None
+
+  /** Scheduler extension services. */
+  private val services: SchedulerExtensionServices = new SchedulerExtensionServices()
+
+  // Flag to specify whether this schedulerBackend should be reset.
+  private var shouldResetOnAmRegister = false
+
+  /**
+   * Bind to YARN. This *must* be done before calling [[start()]].
+   *
+   * @param appId YARN application ID
+   * @param attemptId Optional YARN attempt ID
+   */
+  protected def bindToYarn(appId: ApplicationId, attemptId: Option[ApplicationAttemptId]): Unit = {
+    this.appId = Some(appId)
+    this.attemptId = attemptId
+  }
+
+  override def start() {
+    require(appId.isDefined, "application ID unset")
+    val binding = SchedulerExtensionServiceBinding(sc, appId.get, attemptId)
+    services.start(binding)
+    super.start()
+  }
+
+  override def stop(): Unit = {
+    try {
+      // SPARK-12009: To prevent Yarn allocator from requesting backup for the executors which
+      // was Stopped by SchedulerBackend.
+      requestTotalExecutors(0, 0, Map.empty)
+      super.stop()
+    } finally {
+      services.stop()
+    }
+  }
+
+  /**
+   * Get the attempt ID for this run, if the cluster manager supports multiple
+   * attempts. Applications run in client mode will not have attempt IDs.
+   * This attempt ID only includes attempt counter, like "1", "2".
+   *
+   * @return The application attempt id, if available.
+   */
+  override def applicationAttemptId(): Option[String] = {
+    attemptId.map(_.getAttemptId.toString)
+  }
+
+  /**
+   * Get an application ID associated with the job.
+   * This returns the string value of [[appId]] if set, otherwise
+   * the locally-generated ID from the superclass.
+   * @return The application ID
+   */
+  override def applicationId(): String = {
+    appId.map(_.toString).getOrElse {
+      logWarning("Application ID is not initialized yet.")
+      super.applicationId
+    }
+  }
+
+  /**
+   * Request executors from the ApplicationMaster by specifying the total number desired.
+   * This includes executors already pending or running.
+   */
+  override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = {
+    yarnSchedulerEndpointRef.ask[Boolean](
+      RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount))
+  }
+
+  /**
+   * Request that the ApplicationMaster kill the specified executors.
+   */
+  override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = {
+    yarnSchedulerEndpointRef.ask[Boolean](KillExecutors(executorIds))
+  }
+
+  override def sufficientResourcesRegistered(): Boolean = {
+    totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio
+  }
+
+  /**
+   * Add filters to the SparkUI.
+   */
+  private def addWebUIFilter(
+      filterName: String,
+      filterParams: Map[String, String],
+      proxyBase: String): Unit = {
+    if (proxyBase != null && proxyBase.nonEmpty) {
+      System.setProperty("spark.ui.proxyBase", proxyBase)
+    }
+
+    val hasFilter =
+      filterName != null && filterName.nonEmpty &&
+      filterParams != null && filterParams.nonEmpty
+    if (hasFilter) {
+      logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase")
+      conf.set("spark.ui.filters", filterName)
+      filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) }
+      scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) }
+    }
+  }
+
+  override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = {
+    new YarnDriverEndpoint(rpcEnv, properties)
+  }
+
+  /**
+   * Reset the state of SchedulerBackend to the initial state. This is happened when AM is failed
+   * and re-registered itself to driver after a failure. The stale state in driver should be
+   * cleaned.
+   */
+  override protected def reset(): Unit = {
+    super.reset()
+    sc.executorAllocationManager.foreach(_.reset())
+  }
+
+  /**
+   * Override the DriverEndpoint to add extra logic for the case when an executor is disconnected.
+   * This endpoint communicates with the executors and queries the AM for an executor's exit
+   * status when the executor is disconnected.
+   */
+  private class YarnDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)])
+      extends DriverEndpoint(rpcEnv, sparkProperties) {
+
+    /**
+     * When onDisconnected is received at the driver endpoint, the superclass DriverEndpoint
+     * handles it by assuming the Executor was lost for a bad reason and removes the executor
+     * immediately.
+     *
+     * In YARN's case however it is crucial to talk to the application master and ask why the
+     * executor had exited. If the executor exited for some reason unrelated to the running tasks
+     * (e.g., preemption), according to the application master, then we pass that information down
+     * to the TaskSetManager to inform the TaskSetManager that tasks on that lost executor should
+     * not count towards a job failure.
+     */
+    override def onDisconnected(rpcAddress: RpcAddress): Unit = {
+      addressToExecutorId.get(rpcAddress).foreach { executorId =>
+        if (disableExecutor(executorId)) {
+          yarnSchedulerEndpoint.handleExecutorDisconnectedFromDriver(executorId, rpcAddress)
+        }
+      }
+    }
+  }
+
+  /**
+   * An [[RpcEndpoint]] that communicates with the ApplicationMaster.
+   */
+  private class YarnSchedulerEndpoint(override val rpcEnv: RpcEnv)
+    extends ThreadSafeRpcEndpoint with Logging {
+    private var amEndpoint: Option[RpcEndpointRef] = None
+
+    private[YarnSchedulerBackend] def handleExecutorDisconnectedFromDriver(
+        executorId: String,
+        executorRpcAddress: RpcAddress): Unit = {
+      val removeExecutorMessage = amEndpoint match {
+        case Some(am) =>
+          val lossReasonRequest = GetExecutorLossReason(executorId)
+          am.ask[ExecutorLossReason](lossReasonRequest, askTimeout)
+            .map { reason => RemoveExecutor(executorId, reason) }(ThreadUtils.sameThread)
+            .recover {
+              case NonFatal(e) =>
+                logWarning(s"Attempted to get executor loss reason" +
+                  s" for executor id ${executorId} at RPC address ${executorRpcAddress}," +
+                  s" but got no response. Marking as slave lost.", e)
+                RemoveExecutor(executorId, SlaveLost())
+            }(ThreadUtils.sameThread)
+        case None =>
+          logWarning("Attempted to check for an executor loss reason" +
+            " before the AM has registered!")
+          Future.successful(RemoveExecutor(executorId, SlaveLost("AM is not yet registered.")))
+      }
+
+      removeExecutorMessage
+        .flatMap { message =>
+          driverEndpoint.ask[Boolean](message)
+        }(ThreadUtils.sameThread)
+        .onFailure {
+          case NonFatal(e) => logError(
+            s"Error requesting driver to remove executor $executorId after disconnection.", e)
+        }(ThreadUtils.sameThread)
+    }
+
+    override def receive: PartialFunction[Any, Unit] = {
+      case RegisterClusterManager(am) =>
+        logInfo(s"ApplicationMaster registered as $am")
+        amEndpoint = Option(am)
+        if (!shouldResetOnAmRegister) {
+          shouldResetOnAmRegister = true
+        } else {
+          // AM is already registered before, this potentially means that AM failed and
+          // a new one registered after the failure. This will only happen in yarn-client mode.
+          reset()
+        }
+
+      case AddWebUIFilter(filterName, filterParams, proxyBase) =>
+        addWebUIFilter(filterName, filterParams, proxyBase)
+
+      case r @ RemoveExecutor(executorId, reason) =>
+        logWarning(reason.toString)
+        driverEndpoint.ask[Boolean](r).onFailure {
+          case e =>
+            logError("Error requesting driver to remove executor" +
+              s" $executorId for reason $reason", e)
+        }(ThreadUtils.sameThread)
+    }
+
+
+    override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
+      case r: RequestExecutors =>
+        amEndpoint match {
+          case Some(am) =>
+            am.ask[Boolean](r).andThen {
+              case Success(b) => context.reply(b)
+              case Failure(NonFatal(e)) =>
+                logError(s"Sending $r to AM was unsuccessful", e)
+                context.sendFailure(e)
+            }(ThreadUtils.sameThread)
+          case None =>
+            logWarning("Attempted to request executors before the AM has registered!")
+            context.reply(false)
+        }
+
+      case k: KillExecutors =>
+        amEndpoint match {
+          case Some(am) =>
+            am.ask[Boolean](k).andThen {
+              case Success(b) => context.reply(b)
+              case Failure(NonFatal(e)) =>
+                logError(s"Sending $k to AM was unsuccessful", e)
+                context.sendFailure(e)
+            }(ThreadUtils.sameThread)
+          case None =>
+            logWarning("Attempted to kill executors before the AM has registered!")
+            context.reply(false)
+        }
+
+      case RetrieveLastAllocatedExecutorId =>
+        context.reply(currentExecutorIdCounter)
+    }
+
+    override def onDisconnected(remoteAddress: RpcAddress): Unit = {
+      if (amEndpoint.exists(_.address == remoteAddress)) {
+        logWarning(s"ApplicationMaster has disassociated: $remoteAddress")
+        amEndpoint = None
+      }
+    }
+  }
+}
+
+private[spark] object YarnSchedulerBackend {
+  val ENDPOINT_NAME = "YarnScheduler"
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider b/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
new file mode 100644
index 0000000..d0ef5ef
--- /dev/null
+++ b/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
@@ -0,0 +1 @@
+org.apache.spark.deploy.yarn.security.TestCredentialProvider

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/resources/log4j.properties b/resource-managers/yarn/src/test/resources/log4j.properties
new file mode 100644
index 0000000..d13454d
--- /dev/null
+++ b/resource-managers/yarn/src/test/resources/log4j.properties
@@ -0,0 +1,31 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file target/unit-tests.log
+log4j.rootCategory=DEBUG, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=true
+log4j.appender.file.file=target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
+
+# Ignore messages below warning level from a few verbose libraries.
+log4j.logger.com.sun.jersey=WARN
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.eclipse.jetty=WARN
+log4j.logger.org.mortbay=WARN
+log4j.logger.org.spark_project.jetty=WARN


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


[18/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
[SPARK-18662] Move resource managers to separate directory

## What changes were proposed in this pull request?

* Moves yarn and mesos scheduler backends to resource-managers/ sub-directory (in preparation for https://issues.apache.org/jira/browse/SPARK-18278)
* Corresponding change in top-level pom.xml.

Ref: https://github.com/apache/spark/pull/16061#issuecomment-263649340

## How was this patch tested?

* Manual tests

/cc rxin

Author: Anirudh <ra...@google.com>

Closes #16092 from foxish/fix-scheduler-structure-2.


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

Branch: refs/heads/master
Commit: 81e5619ca141a1d3a06547d2b682cbe3f135b360
Parents: a8ced76
Author: Anirudh <ra...@google.com>
Authored: Tue Dec 6 16:23:27 2016 -0800
Committer: Marcelo Vanzin <va...@cloudera.com>
Committed: Tue Dec 6 16:23:27 2016 -0800

----------------------------------------------------------------------
 dev/sparktestsupport/modules.py                 |    4 +-
 mesos/pom.xml                                   |  109 --
 ...pache.spark.scheduler.ExternalClusterManager |    1 -
 .../deploy/mesos/MesosClusterDispatcher.scala   |  119 --
 .../mesos/MesosClusterDispatcherArguments.scala |  149 --
 .../deploy/mesos/MesosDriverDescription.scala   |   70 -
 .../mesos/MesosExternalShuffleService.scala     |  131 --
 .../org/apache/spark/deploy/mesos/config.scala  |   59 -
 .../spark/deploy/mesos/ui/DriverPage.scala      |  179 --
 .../deploy/mesos/ui/MesosClusterPage.scala      |  136 --
 .../spark/deploy/mesos/ui/MesosClusterUI.scala  |   49 -
 .../deploy/rest/mesos/MesosRestServer.scala     |  156 --
 .../spark/executor/MesosExecutorBackend.scala   |  131 --
 .../cluster/mesos/MesosClusterManager.scala     |   64 -
 .../mesos/MesosClusterPersistenceEngine.scala   |  134 --
 .../cluster/mesos/MesosClusterScheduler.scala   |  740 ---------
 .../mesos/MesosClusterSchedulerSource.scala     |   40 -
 .../MesosCoarseGrainedSchedulerBackend.scala    |  668 --------
 .../MesosFineGrainedSchedulerBackend.scala      |  444 -----
 .../mesos/MesosSchedulerBackendUtil.scala       |  165 --
 .../cluster/mesos/MesosSchedulerUtils.scala     |  524 ------
 .../cluster/mesos/MesosTaskLaunchData.scala     |   51 -
 .../MesosClusterDispatcherArgumentsSuite.scala  |   63 -
 .../mesos/MesosClusterDispatcherSuite.scala     |   40 -
 .../mesos/MesosClusterManagerSuite.scala        |   56 -
 .../mesos/MesosClusterSchedulerSuite.scala      |  239 ---
 ...esosCoarseGrainedSchedulerBackendSuite.scala |  601 -------
 .../MesosFineGrainedSchedulerBackendSuite.scala |  385 -----
 .../mesos/MesosSchedulerUtilsSuite.scala        |  256 ---
 .../mesos/MesosTaskLaunchDataSuite.scala        |   36 -
 .../spark/scheduler/cluster/mesos/Utils.scala   |   91 --
 pom.xml                                         |    4 +-
 resource-managers/mesos/pom.xml                 |  109 ++
 ...pache.spark.scheduler.ExternalClusterManager |    1 +
 .../deploy/mesos/MesosClusterDispatcher.scala   |  119 ++
 .../mesos/MesosClusterDispatcherArguments.scala |  149 ++
 .../deploy/mesos/MesosDriverDescription.scala   |   70 +
 .../mesos/MesosExternalShuffleService.scala     |  131 ++
 .../org/apache/spark/deploy/mesos/config.scala  |   59 +
 .../spark/deploy/mesos/ui/DriverPage.scala      |  179 ++
 .../deploy/mesos/ui/MesosClusterPage.scala      |  136 ++
 .../spark/deploy/mesos/ui/MesosClusterUI.scala  |   49 +
 .../deploy/rest/mesos/MesosRestServer.scala     |  156 ++
 .../spark/executor/MesosExecutorBackend.scala   |  131 ++
 .../cluster/mesos/MesosClusterManager.scala     |   64 +
 .../mesos/MesosClusterPersistenceEngine.scala   |  134 ++
 .../cluster/mesos/MesosClusterScheduler.scala   |  740 +++++++++
 .../mesos/MesosClusterSchedulerSource.scala     |   40 +
 .../MesosCoarseGrainedSchedulerBackend.scala    |  668 ++++++++
 .../MesosFineGrainedSchedulerBackend.scala      |  444 +++++
 .../mesos/MesosSchedulerBackendUtil.scala       |  165 ++
 .../cluster/mesos/MesosSchedulerUtils.scala     |  524 ++++++
 .../cluster/mesos/MesosTaskLaunchData.scala     |   51 +
 .../MesosClusterDispatcherArgumentsSuite.scala  |   63 +
 .../mesos/MesosClusterDispatcherSuite.scala     |   40 +
 .../mesos/MesosClusterManagerSuite.scala        |   56 +
 .../mesos/MesosClusterSchedulerSuite.scala      |  239 +++
 ...esosCoarseGrainedSchedulerBackendSuite.scala |  601 +++++++
 .../MesosFineGrainedSchedulerBackendSuite.scala |  385 +++++
 .../mesos/MesosSchedulerUtilsSuite.scala        |  256 +++
 .../mesos/MesosTaskLaunchDataSuite.scala        |   36 +
 .../spark/scheduler/cluster/mesos/Utils.scala   |   91 ++
 resource-managers/yarn/pom.xml                  |  215 +++
 ...ploy.yarn.security.ServiceCredentialProvider |    3 +
 ...pache.spark.scheduler.ExternalClusterManager |    1 +
 .../spark/deploy/yarn/ApplicationMaster.scala   |  791 +++++++++
 .../yarn/ApplicationMasterArguments.scala       |  105 ++
 .../org/apache/spark/deploy/yarn/Client.scala   | 1541 ++++++++++++++++++
 .../spark/deploy/yarn/ClientArguments.scala     |   86 +
 .../yarn/ClientDistributedCacheManager.scala    |  186 +++
 .../spark/deploy/yarn/ExecutorRunnable.scala    |  266 +++
 ...ityPreferredContainerPlacementStrategy.scala |  224 +++
 .../spark/deploy/yarn/YarnAllocator.scala       |  727 +++++++++
 .../apache/spark/deploy/yarn/YarnRMClient.scala |  135 ++
 .../spark/deploy/yarn/YarnSparkHadoopUtil.scala |  317 ++++
 .../org/apache/spark/deploy/yarn/config.scala   |  347 ++++
 .../yarn/security/AMCredentialRenewer.scala     |  235 +++
 .../ConfigurableCredentialManager.scala         |  105 ++
 .../yarn/security/CredentialUpdater.scala       |  130 ++
 .../yarn/security/HBaseCredentialProvider.scala |   74 +
 .../yarn/security/HDFSCredentialProvider.scala  |  110 ++
 .../yarn/security/HiveCredentialProvider.scala  |  129 ++
 .../security/ServiceCredentialProvider.scala    |   57 +
 .../launcher/YarnCommandBuilderUtils.scala      |   53 +
 .../cluster/SchedulerExtensionService.scala     |  143 ++
 .../cluster/YarnClientSchedulerBackend.scala    |  157 ++
 .../scheduler/cluster/YarnClusterManager.scala  |   56 +
 .../cluster/YarnClusterScheduler.scala          |   37 +
 .../cluster/YarnClusterSchedulerBackend.scala   |   67 +
 .../spark/scheduler/cluster/YarnScheduler.scala |   39 +
 .../cluster/YarnSchedulerBackend.scala          |  315 ++++
 ...ploy.yarn.security.ServiceCredentialProvider |    1 +
 .../yarn/src/test/resources/log4j.properties    |   31 +
 .../deploy/yarn/BaseYarnClusterSuite.scala      |  241 +++
 .../ClientDistributedCacheManagerSuite.scala    |  204 +++
 .../apache/spark/deploy/yarn/ClientSuite.scala  |  462 ++++++
 .../yarn/ContainerPlacementStrategySuite.scala  |  153 ++
 .../spark/deploy/yarn/YarnAllocatorSuite.scala  |  344 ++++
 .../spark/deploy/yarn/YarnClusterSuite.scala    |  493 ++++++
 .../yarn/YarnShuffleIntegrationSuite.scala      |  112 ++
 .../deploy/yarn/YarnSparkHadoopUtilSuite.scala  |  213 +++
 .../ConfigurableCredentialManagerSuite.scala    |  150 ++
 .../security/HDFSCredentialProviderSuite.scala  |   71 +
 .../spark/launcher/TestClasspathBuilder.scala   |   36 +
 .../network/shuffle/ShuffleTestAccessor.scala   |   70 +
 .../network/yarn/YarnShuffleServiceSuite.scala  |  372 +++++
 .../spark/network/yarn/YarnTestAccessor.scala   |   37 +
 .../ExtensionServiceIntegrationSuite.scala      |   72 +
 .../cluster/SimpleExtensionService.scala        |   34 +
 .../cluster/StubApplicationAttemptId.scala      |   48 +
 .../scheduler/cluster/StubApplicationId.scala   |   42 +
 yarn/pom.xml                                    |  215 ---
 ...ploy.yarn.security.ServiceCredentialProvider |    3 -
 ...pache.spark.scheduler.ExternalClusterManager |    1 -
 .../spark/deploy/yarn/ApplicationMaster.scala   |  791 ---------
 .../yarn/ApplicationMasterArguments.scala       |  105 --
 .../org/apache/spark/deploy/yarn/Client.scala   | 1541 ------------------
 .../spark/deploy/yarn/ClientArguments.scala     |   86 -
 .../yarn/ClientDistributedCacheManager.scala    |  186 ---
 .../spark/deploy/yarn/ExecutorRunnable.scala    |  266 ---
 ...ityPreferredContainerPlacementStrategy.scala |  224 ---
 .../spark/deploy/yarn/YarnAllocator.scala       |  727 ---------
 .../apache/spark/deploy/yarn/YarnRMClient.scala |  135 --
 .../spark/deploy/yarn/YarnSparkHadoopUtil.scala |  317 ----
 .../org/apache/spark/deploy/yarn/config.scala   |  347 ----
 .../yarn/security/AMCredentialRenewer.scala     |  235 ---
 .../ConfigurableCredentialManager.scala         |  105 --
 .../yarn/security/CredentialUpdater.scala       |  130 --
 .../yarn/security/HBaseCredentialProvider.scala |   74 -
 .../yarn/security/HDFSCredentialProvider.scala  |  110 --
 .../yarn/security/HiveCredentialProvider.scala  |  129 --
 .../security/ServiceCredentialProvider.scala    |   57 -
 .../launcher/YarnCommandBuilderUtils.scala      |   53 -
 .../cluster/SchedulerExtensionService.scala     |  143 --
 .../cluster/YarnClientSchedulerBackend.scala    |  157 --
 .../scheduler/cluster/YarnClusterManager.scala  |   56 -
 .../cluster/YarnClusterScheduler.scala          |   37 -
 .../cluster/YarnClusterSchedulerBackend.scala   |   67 -
 .../spark/scheduler/cluster/YarnScheduler.scala |   39 -
 .../cluster/YarnSchedulerBackend.scala          |  315 ----
 ...ploy.yarn.security.ServiceCredentialProvider |    1 -
 yarn/src/test/resources/log4j.properties        |   31 -
 .../deploy/yarn/BaseYarnClusterSuite.scala      |  241 ---
 .../ClientDistributedCacheManagerSuite.scala    |  204 ---
 .../apache/spark/deploy/yarn/ClientSuite.scala  |  462 ------
 .../yarn/ContainerPlacementStrategySuite.scala  |  153 --
 .../spark/deploy/yarn/YarnAllocatorSuite.scala  |  344 ----
 .../spark/deploy/yarn/YarnClusterSuite.scala    |  493 ------
 .../yarn/YarnShuffleIntegrationSuite.scala      |  112 --
 .../deploy/yarn/YarnSparkHadoopUtilSuite.scala  |  213 ---
 .../ConfigurableCredentialManagerSuite.scala    |  150 --
 .../security/HDFSCredentialProviderSuite.scala  |   71 -
 .../spark/launcher/TestClasspathBuilder.scala   |   36 -
 .../network/shuffle/ShuffleTestAccessor.scala   |   70 -
 .../network/yarn/YarnShuffleServiceSuite.scala  |  372 -----
 .../spark/network/yarn/YarnTestAccessor.scala   |   37 -
 .../ExtensionServiceIntegrationSuite.scala      |   72 -
 .../cluster/SimpleExtensionService.scala        |   34 -
 .../cluster/StubApplicationAttemptId.scala      |   48 -
 .../scheduler/cluster/StubApplicationId.scala   |   42 -
 160 files changed, 15727 insertions(+), 15727 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/dev/sparktestsupport/modules.py
----------------------------------------------------------------------
diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py
index b34ab51..1a7cf9a 100644
--- a/dev/sparktestsupport/modules.py
+++ b/dev/sparktestsupport/modules.py
@@ -469,7 +469,7 @@ yarn = Module(
     name="yarn",
     dependencies=[],
     source_file_regexes=[
-        "yarn/",
+        "resource-managers/yarn/",
         "common/network-yarn/",
     ],
     build_profile_flags=["-Pyarn"],
@@ -485,7 +485,7 @@ yarn = Module(
 mesos = Module(
     name="mesos",
     dependencies=[],
-    source_file_regexes=["mesos/"],
+    source_file_regexes=["resource-managers/mesos/"],
     build_profile_flags=["-Pmesos"],
     sbt_test_goals=["mesos/test"]
 )

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/mesos/pom.xml b/mesos/pom.xml
deleted file mode 100644
index 60f2359..0000000
--- a/mesos/pom.xml
+++ /dev/null
@@ -1,109 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one or more
-  ~ contributor license agreements.  See the NOTICE file distributed with
-  ~ this work for additional information regarding copyright ownership.
-  ~ The ASF licenses this file to You under the Apache License, Version 2.0
-  ~ (the "License"); you may not use this file except in compliance with
-  ~ the License.  You may obtain a copy of the License at
-  ~
-  ~    http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS,
-  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~ See the License for the specific language governing permissions and
-  ~ limitations under the License.
-  -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.spark</groupId>
-    <artifactId>spark-parent_2.11</artifactId>
-    <version>2.2.0-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
-
-  <artifactId>spark-mesos_2.11</artifactId>
-  <packaging>jar</packaging>
-  <name>Spark Project Mesos</name>
-  <properties>
-    <sbt.project.name>mesos</sbt.project.name>
-    <mesos.version>1.0.0</mesos.version>
-    <mesos.classifier>shaded-protobuf</mesos.classifier>
-  </properties>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.mesos</groupId>
-      <artifactId>mesos</artifactId>
-      <version>${mesos.version}</version>
-      <classifier>${mesos.classifier}</classifier>
-      <exclusions>
-        <exclusion>
-          <groupId>com.google.protobuf</groupId>
-          <artifactId>protobuf-java</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-core</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <!-- Explicitly depend on shaded dependencies from the parent, since shaded deps aren't transitive -->
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-server</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-plus</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-http</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-servlet</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-servlets</artifactId>
-    </dependency>
-    <!-- End of shaded deps. -->
-
-  </dependencies>
-
-
-  <build>
-    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
-  </build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
----------------------------------------------------------------------
diff --git a/mesos/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/mesos/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
deleted file mode 100644
index 12b6d5b..0000000
--- a/mesos/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
+++ /dev/null
@@ -1 +0,0 @@
-org.apache.spark.scheduler.cluster.mesos.MesosClusterManager

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala
deleted file mode 100644
index 792ade8..0000000
--- a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala
+++ /dev/null
@@ -1,119 +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.deploy.mesos
-
-import java.util.concurrent.CountDownLatch
-
-import org.apache.spark.{SecurityManager, SparkConf}
-import org.apache.spark.deploy.mesos.config._
-import org.apache.spark.deploy.mesos.ui.MesosClusterUI
-import org.apache.spark.deploy.rest.mesos.MesosRestServer
-import org.apache.spark.internal.Logging
-import org.apache.spark.scheduler.cluster.mesos._
-import org.apache.spark.util.{CommandLineUtils, ShutdownHookManager, Utils}
-
-/*
- * A dispatcher that is responsible for managing and launching drivers, and is intended to be
- * used for Mesos cluster mode. The dispatcher is a long-running process started by the user in
- * the cluster independently of Spark applications.
- * It contains a [[MesosRestServer]] that listens for requests to submit drivers and a
- * [[MesosClusterScheduler]] that processes these requests by negotiating with the Mesos master
- * for resources.
- *
- * A typical new driver lifecycle is the following:
- * - Driver submitted via spark-submit talking to the [[MesosRestServer]]
- * - [[MesosRestServer]] queues the driver request to [[MesosClusterScheduler]]
- * - [[MesosClusterScheduler]] gets resource offers and launches the drivers that are in queue
- *
- * This dispatcher supports both Mesos fine-grain or coarse-grain mode as the mode is configurable
- * per driver launched.
- * This class is needed since Mesos doesn't manage frameworks, so the dispatcher acts as
- * a daemon to launch drivers as Mesos frameworks upon request. The dispatcher is also started and
- * stopped by sbin/start-mesos-dispatcher and sbin/stop-mesos-dispatcher respectively.
- */
-private[mesos] class MesosClusterDispatcher(
-    args: MesosClusterDispatcherArguments,
-    conf: SparkConf)
-  extends Logging {
-
-  private val publicAddress = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(args.host)
-  private val recoveryMode = conf.get(RECOVERY_MODE).toUpperCase()
-  logInfo("Recovery mode in Mesos dispatcher set to: " + recoveryMode)
-
-  private val engineFactory = recoveryMode match {
-    case "NONE" => new BlackHoleMesosClusterPersistenceEngineFactory
-    case "ZOOKEEPER" => new ZookeeperMesosClusterPersistenceEngineFactory(conf)
-    case _ => throw new IllegalArgumentException("Unsupported recovery mode: " + recoveryMode)
-  }
-
-  private val scheduler = new MesosClusterScheduler(engineFactory, conf)
-
-  private val server = new MesosRestServer(args.host, args.port, conf, scheduler)
-  private val webUi = new MesosClusterUI(
-    new SecurityManager(conf),
-    args.webUiPort,
-    conf,
-    publicAddress,
-    scheduler)
-
-  private val shutdownLatch = new CountDownLatch(1)
-
-  def start(): Unit = {
-    webUi.bind()
-    scheduler.frameworkUrl = conf.get(DISPATCHER_WEBUI_URL).getOrElse(webUi.activeWebUiUrl)
-    scheduler.start()
-    server.start()
-  }
-
-  def awaitShutdown(): Unit = {
-    shutdownLatch.await()
-  }
-
-  def stop(): Unit = {
-    webUi.stop()
-    server.stop()
-    scheduler.stop()
-    shutdownLatch.countDown()
-  }
-}
-
-private[mesos] object MesosClusterDispatcher
-  extends Logging
-  with CommandLineUtils {
-
-  override def main(args: Array[String]) {
-    Utils.initDaemon(log)
-    val conf = new SparkConf
-    val dispatcherArgs = new MesosClusterDispatcherArguments(args, conf)
-    conf.setMaster(dispatcherArgs.masterUrl)
-    conf.setAppName(dispatcherArgs.name)
-    dispatcherArgs.zookeeperUrl.foreach { z =>
-      conf.set(RECOVERY_MODE, "ZOOKEEPER")
-      conf.set(ZOOKEEPER_URL, z)
-    }
-    val dispatcher = new MesosClusterDispatcher(dispatcherArgs, conf)
-    dispatcher.start()
-    logDebug("Adding shutdown hook") // force eager creation of logger
-    ShutdownHookManager.addShutdownHook { () =>
-      logInfo("Shutdown hook is shutting down dispatcher")
-      dispatcher.stop()
-      dispatcher.awaitShutdown()
-    }
-    dispatcher.awaitShutdown()
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
deleted file mode 100644
index ef08502..0000000
--- a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
+++ /dev/null
@@ -1,149 +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.deploy.mesos
-
-import scala.annotation.tailrec
-import scala.collection.mutable
-
-import org.apache.spark.util.{IntParam, Utils}
-import org.apache.spark.SparkConf
-
-private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf: SparkConf) {
-  var host: String = Utils.localHostName()
-  var port: Int = 7077
-  var name: String = "Spark Cluster"
-  var webUiPort: Int = 8081
-  var verbose: Boolean = false
-  var masterUrl: String = _
-  var zookeeperUrl: Option[String] = None
-  var propertiesFile: String = _
-  val confProperties: mutable.HashMap[String, String] =
-    new mutable.HashMap[String, String]()
-
-  parse(args.toList)
-
-  // scalastyle:on println
-  propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile)
-  Utils.updateSparkConfigFromProperties(conf, confProperties)
-
-  // scalastyle:off println
-  if (verbose) {
-    MesosClusterDispatcher.printStream.println(s"Using host: $host")
-    MesosClusterDispatcher.printStream.println(s"Using port: $port")
-    MesosClusterDispatcher.printStream.println(s"Using webUiPort: $webUiPort")
-    MesosClusterDispatcher.printStream.println(s"Framework Name: $name")
-
-    Option(propertiesFile).foreach { file =>
-      MesosClusterDispatcher.printStream.println(s"Using properties file: $file")
-    }
-
-    MesosClusterDispatcher.printStream.println(s"Spark Config properties set:")
-    conf.getAll.foreach(println)
-  }
-
-  @tailrec
-  private def parse(args: List[String]): Unit = args match {
-    case ("--host" | "-h") :: value :: tail =>
-      Utils.checkHost(value, "Please use hostname " + value)
-      host = value
-      parse(tail)
-
-    case ("--port" | "-p") :: IntParam(value) :: tail =>
-      port = value
-      parse(tail)
-
-    case ("--webui-port") :: IntParam(value) :: tail =>
-      webUiPort = value
-      parse(tail)
-
-    case ("--zk" | "-z") :: value :: tail =>
-      zookeeperUrl = Some(value)
-      parse(tail)
-
-    case ("--master" | "-m") :: value :: tail =>
-      if (!value.startsWith("mesos://")) {
-        // scalastyle:off println
-        MesosClusterDispatcher.printStream
-          .println("Cluster dispatcher only supports mesos (uri begins with mesos://)")
-        // scalastyle:on println
-        MesosClusterDispatcher.exitFn(1)
-      }
-      masterUrl = value.stripPrefix("mesos://")
-      parse(tail)
-
-    case ("--name") :: value :: tail =>
-      name = value
-      parse(tail)
-
-    case ("--properties-file") :: value :: tail =>
-      propertiesFile = value
-      parse(tail)
-
-    case ("--conf") :: value :: tail =>
-      val pair = MesosClusterDispatcher.
-        parseSparkConfProperty(value)
-        confProperties(pair._1) = pair._2
-      parse(tail)
-
-    case ("--help") :: tail =>
-        printUsageAndExit(0)
-
-    case ("--verbose") :: tail =>
-      verbose = true
-      parse(tail)
-
-    case Nil =>
-      if (Option(masterUrl).isEmpty) {
-        // scalastyle:off println
-        MesosClusterDispatcher.printStream.println("--master is required")
-        // scalastyle:on println
-        printUsageAndExit(1)
-      }
-
-    case value =>
-      // scalastyle:off println
-      MesosClusterDispatcher.printStream.println(s"Unrecognized option: '${value.head}'")
-      // scalastyle:on println
-      printUsageAndExit(1)
-  }
-
-  private def printUsageAndExit(exitCode: Int): Unit = {
-    val outStream = MesosClusterDispatcher.printStream
-
-    // scalastyle:off println
-    outStream.println(
-      "Usage: MesosClusterDispatcher [options]\n" +
-        "\n" +
-        "Options:\n" +
-        "  -h HOST, --host HOST    Hostname to listen on\n" +
-        "  --help                  Show this help message and exit.\n" +
-        "  --verbose,              Print additional debug output.\n" +
-        "  -p PORT, --port PORT    Port to listen on (default: 7077)\n" +
-        "  --webui-port WEBUI_PORT WebUI Port to listen on (default: 8081)\n" +
-        "  --name NAME             Framework name to show in Mesos UI\n" +
-        "  -m --master MASTER      URI for connecting to Mesos master\n" +
-        "  -z --zk ZOOKEEPER       Comma delimited URLs for connecting to \n" +
-        "                          Zookeeper for persistence\n" +
-        "  --properties-file FILE  Path to a custom Spark properties file.\n" +
-        "                          Default is conf/spark-defaults.conf \n" +
-        "  --conf PROP=VALUE       Arbitrary Spark configuration property.\n" +
-        "                          Takes precedence over defined properties in properties-file.")
-    // scalastyle:on println
-    MesosClusterDispatcher.exitFn(exitCode)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala
deleted file mode 100644
index d4c7022..0000000
--- a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala
+++ /dev/null
@@ -1,70 +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.deploy.mesos
-
-import java.util.Date
-
-import org.apache.spark.SparkConf
-import org.apache.spark.deploy.Command
-import org.apache.spark.scheduler.cluster.mesos.MesosClusterRetryState
-
-/**
- * Describes a Spark driver that is submitted from the
- * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]], to be launched by
- * [[org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler]].
- * @param jarUrl URL to the application jar
- * @param mem Amount of memory for the driver
- * @param cores Number of cores for the driver
- * @param supervise Supervise the driver for long running app
- * @param command The command to launch the driver.
- * @param schedulerProperties Extra properties to pass the Mesos scheduler
- */
-private[spark] class MesosDriverDescription(
-    val name: String,
-    val jarUrl: String,
-    val mem: Int,
-    val cores: Double,
-    val supervise: Boolean,
-    val command: Command,
-    schedulerProperties: Map[String, String],
-    val submissionId: String,
-    val submissionDate: Date,
-    val retryState: Option[MesosClusterRetryState] = None)
-  extends Serializable {
-
-  val conf = new SparkConf(false)
-  schedulerProperties.foreach {case (k, v) => conf.set(k, v)}
-
-  def copy(
-      name: String = name,
-      jarUrl: String = jarUrl,
-      mem: Int = mem,
-      cores: Double = cores,
-      supervise: Boolean = supervise,
-      command: Command = command,
-      schedulerProperties: SparkConf = conf,
-      submissionId: String = submissionId,
-      submissionDate: Date = submissionDate,
-      retryState: Option[MesosClusterRetryState] = retryState): MesosDriverDescription = {
-
-    new MesosDriverDescription(name, jarUrl, mem, cores, supervise, command, conf.getAll.toMap,
-      submissionId, submissionDate, retryState)
-  }
-
-  override def toString: String = s"MesosDriverDescription (${command.mainClass})"
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala
deleted file mode 100644
index 859aa83..0000000
--- a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala
+++ /dev/null
@@ -1,131 +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.deploy.mesos
-
-import java.nio.ByteBuffer
-import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.{SecurityManager, SparkConf}
-import org.apache.spark.deploy.ExternalShuffleService
-import org.apache.spark.deploy.mesos.config._
-import org.apache.spark.internal.Logging
-import org.apache.spark.network.client.{RpcResponseCallback, TransportClient}
-import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler
-import org.apache.spark.network.shuffle.protocol.BlockTransferMessage
-import org.apache.spark.network.shuffle.protocol.mesos.{RegisterDriver, ShuffleServiceHeartbeat}
-import org.apache.spark.network.util.TransportConf
-import org.apache.spark.util.ThreadUtils
-
-/**
- * An RPC endpoint that receives registration requests from Spark drivers running on Mesos.
- * It detects driver termination and calls the cleanup callback to [[ExternalShuffleService]].
- */
-private[mesos] class MesosExternalShuffleBlockHandler(
-    transportConf: TransportConf,
-    cleanerIntervalS: Long)
-  extends ExternalShuffleBlockHandler(transportConf, null) with Logging {
-
-  ThreadUtils.newDaemonSingleThreadScheduledExecutor("shuffle-cleaner-watcher")
-    .scheduleAtFixedRate(new CleanerThread(), 0, cleanerIntervalS, TimeUnit.SECONDS)
-
-  // Stores a map of app id to app state (timeout value and last heartbeat)
-  private val connectedApps = new ConcurrentHashMap[String, AppState]()
-
-  protected override def handleMessage(
-      message: BlockTransferMessage,
-      client: TransportClient,
-      callback: RpcResponseCallback): Unit = {
-    message match {
-      case RegisterDriverParam(appId, appState) =>
-        val address = client.getSocketAddress
-        val timeout = appState.heartbeatTimeout
-        logInfo(s"Received registration request from app $appId (remote address $address, " +
-          s"heartbeat timeout $timeout ms).")
-        if (connectedApps.containsKey(appId)) {
-          logWarning(s"Received a registration request from app $appId, but it was already " +
-            s"registered")
-        }
-        connectedApps.put(appId, appState)
-        callback.onSuccess(ByteBuffer.allocate(0))
-      case Heartbeat(appId) =>
-        val address = client.getSocketAddress
-        Option(connectedApps.get(appId)) match {
-          case Some(existingAppState) =>
-            logTrace(s"Received ShuffleServiceHeartbeat from app '$appId' (remote " +
-              s"address $address).")
-            existingAppState.lastHeartbeat = System.nanoTime()
-          case None =>
-            logWarning(s"Received ShuffleServiceHeartbeat from an unknown app (remote " +
-              s"address $address, appId '$appId').")
-        }
-      case _ => super.handleMessage(message, client, callback)
-    }
-  }
-
-  /** An extractor object for matching [[RegisterDriver]] message. */
-  private object RegisterDriverParam {
-    def unapply(r: RegisterDriver): Option[(String, AppState)] =
-      Some((r.getAppId, new AppState(r.getHeartbeatTimeoutMs, System.nanoTime())))
-  }
-
-  private object Heartbeat {
-    def unapply(h: ShuffleServiceHeartbeat): Option[String] = Some(h.getAppId)
-  }
-
-  private class AppState(val heartbeatTimeout: Long, @volatile var lastHeartbeat: Long)
-
-  private class CleanerThread extends Runnable {
-    override def run(): Unit = {
-      val now = System.nanoTime()
-      connectedApps.asScala.foreach { case (appId, appState) =>
-        if (now - appState.lastHeartbeat > appState.heartbeatTimeout * 1000 * 1000) {
-          logInfo(s"Application $appId timed out. Removing shuffle files.")
-          connectedApps.remove(appId)
-          applicationRemoved(appId, true)
-        }
-      }
-    }
-  }
-}
-
-/**
- * A wrapper of [[ExternalShuffleService]] that provides an additional endpoint for drivers
- * to associate with. This allows the shuffle service to detect when a driver is terminated
- * and can clean up the associated shuffle files.
- */
-private[mesos] class MesosExternalShuffleService(conf: SparkConf, securityManager: SecurityManager)
-  extends ExternalShuffleService(conf, securityManager) {
-
-  protected override def newShuffleBlockHandler(
-      conf: TransportConf): ExternalShuffleBlockHandler = {
-    val cleanerIntervalS = this.conf.get(SHUFFLE_CLEANER_INTERVAL_S)
-    new MesosExternalShuffleBlockHandler(conf, cleanerIntervalS)
-  }
-}
-
-private[spark] object MesosExternalShuffleService extends Logging {
-
-  def main(args: Array[String]): Unit = {
-    ExternalShuffleService.main(args,
-      (conf: SparkConf, sm: SecurityManager) => new MesosExternalShuffleService(conf, sm))
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala
deleted file mode 100644
index 19e2533..0000000
--- a/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala
+++ /dev/null
@@ -1,59 +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.deploy.mesos
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.spark.internal.config.ConfigBuilder
-
-package object config {
-
-  /* Common app configuration. */
-
-  private[spark] val SHUFFLE_CLEANER_INTERVAL_S =
-    ConfigBuilder("spark.shuffle.cleaner.interval")
-      .timeConf(TimeUnit.SECONDS)
-      .createWithDefaultString("30s")
-
-  private[spark] val RECOVERY_MODE =
-    ConfigBuilder("spark.deploy.recoveryMode")
-      .stringConf
-      .createWithDefault("NONE")
-
-  private[spark] val DISPATCHER_WEBUI_URL =
-    ConfigBuilder("spark.mesos.dispatcher.webui.url")
-      .doc("Set the Spark Mesos dispatcher webui_url for interacting with the " +
-        "framework. If unset it will point to Spark's internal web UI.")
-      .stringConf
-      .createOptional
-
-  private[spark] val ZOOKEEPER_URL =
-    ConfigBuilder("spark.deploy.zookeeper.url")
-      .doc("When `spark.deploy.recoveryMode` is set to ZOOKEEPER, this " +
-        "configuration is used to set the zookeeper URL to connect to.")
-      .stringConf
-      .createOptional
-
-  private[spark] val HISTORY_SERVER_URL =
-    ConfigBuilder("spark.mesos.dispatcher.historyServer.url")
-      .doc("Set the URL of the history server. The dispatcher will then " +
-        "link each driver to its entry in the history server.")
-      .stringConf
-      .createOptional
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala
deleted file mode 100644
index cd98110..0000000
--- a/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala
+++ /dev/null
@@ -1,179 +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.deploy.mesos.ui
-
-import javax.servlet.http.HttpServletRequest
-
-import scala.xml.Node
-
-import org.apache.spark.deploy.Command
-import org.apache.spark.deploy.mesos.MesosDriverDescription
-import org.apache.spark.scheduler.cluster.mesos.{MesosClusterRetryState, MesosClusterSubmissionState}
-import org.apache.spark.ui.{UIUtils, WebUIPage}
-
-private[ui] class DriverPage(parent: MesosClusterUI) extends WebUIPage("driver") {
-
-  override def render(request: HttpServletRequest): Seq[Node] = {
-    val driverId = request.getParameter("id")
-    require(driverId != null && driverId.nonEmpty, "Missing id parameter")
-
-    val state = parent.scheduler.getDriverState(driverId)
-    if (state.isEmpty) {
-      val content =
-        <div>
-          <p>Cannot find driver {driverId}</p>
-        </div>
-      return UIUtils.basicSparkPage(content, s"Details for Job $driverId")
-    }
-    val driverState = state.get
-    val driverHeaders = Seq("Driver property", "Value")
-    val schedulerHeaders = Seq("Scheduler property", "Value")
-    val commandEnvHeaders = Seq("Command environment variable", "Value")
-    val launchedHeaders = Seq("Launched property", "Value")
-    val commandHeaders = Seq("Command property", "Value")
-    val retryHeaders = Seq("Last failed status", "Next retry time", "Retry count")
-    val driverDescription = Iterable.apply(driverState.description)
-    val submissionState = Iterable.apply(driverState.submissionState)
-    val command = Iterable.apply(driverState.description.command)
-    val schedulerProperties = Iterable.apply(driverState.description.conf.getAll.toMap)
-    val commandEnv = Iterable.apply(driverState.description.command.environment)
-    val driverTable =
-      UIUtils.listingTable(driverHeaders, driverRow, driverDescription)
-    val commandTable =
-      UIUtils.listingTable(commandHeaders, commandRow, command)
-    val commandEnvTable =
-      UIUtils.listingTable(commandEnvHeaders, propertiesRow, commandEnv)
-    val schedulerTable =
-      UIUtils.listingTable(schedulerHeaders, propertiesRow, schedulerProperties)
-    val launchedTable =
-      UIUtils.listingTable(launchedHeaders, launchedRow, submissionState)
-    val retryTable =
-      UIUtils.listingTable(
-        retryHeaders, retryRow, Iterable.apply(driverState.description.retryState))
-    val content =
-      <p>Driver state information for driver id {driverId}</p>
-        <a href={UIUtils.prependBaseUri("/")}>Back to Drivers</a>
-        <div class="row-fluid">
-          <div class="span12">
-            <h4>Driver state: {driverState.state}</h4>
-            <h4>Driver properties</h4>
-            {driverTable}
-            <h4>Driver command</h4>
-            {commandTable}
-            <h4>Driver command environment</h4>
-            {commandEnvTable}
-            <h4>Scheduler properties</h4>
-            {schedulerTable}
-            <h4>Launched state</h4>
-            {launchedTable}
-            <h4>Retry state</h4>
-            {retryTable}
-          </div>
-        </div>;
-
-    UIUtils.basicSparkPage(content, s"Details for Job $driverId")
-  }
-
-  private def launchedRow(submissionState: Option[MesosClusterSubmissionState]): Seq[Node] = {
-    submissionState.map { state =>
-      <tr>
-        <td>Mesos Slave ID</td>
-        <td>{state.slaveId.getValue}</td>
-      </tr>
-      <tr>
-        <td>Mesos Task ID</td>
-        <td>{state.taskId.getValue}</td>
-      </tr>
-      <tr>
-        <td>Launch Time</td>
-        <td>{state.startDate}</td>
-      </tr>
-      <tr>
-        <td>Finish Time</td>
-        <td>{state.finishDate.map(_.toString).getOrElse("")}</td>
-      </tr>
-      <tr>
-        <td>Last Task Status</td>
-        <td>{state.mesosTaskStatus.map(_.toString).getOrElse("")}</td>
-      </tr>
-    }.getOrElse(Seq[Node]())
-  }
-
-  private def propertiesRow(properties: collection.Map[String, String]): Seq[Node] = {
-    properties.map { case (k, v) =>
-      <tr>
-        <td>{k}</td><td>{v}</td>
-      </tr>
-    }.toSeq
-  }
-
-  private def commandRow(command: Command): Seq[Node] = {
-    <tr>
-      <td>Main class</td><td>{command.mainClass}</td>
-    </tr>
-    <tr>
-      <td>Arguments</td><td>{command.arguments.mkString(" ")}</td>
-    </tr>
-    <tr>
-      <td>Class path entries</td><td>{command.classPathEntries.mkString(" ")}</td>
-    </tr>
-    <tr>
-      <td>Java options</td><td>{command.javaOpts.mkString((" "))}</td>
-    </tr>
-    <tr>
-      <td>Library path entries</td><td>{command.libraryPathEntries.mkString((" "))}</td>
-    </tr>
-  }
-
-  private def driverRow(driver: MesosDriverDescription): Seq[Node] = {
-    <tr>
-      <td>Name</td><td>{driver.name}</td>
-    </tr>
-    <tr>
-      <td>Id</td><td>{driver.submissionId}</td>
-    </tr>
-    <tr>
-      <td>Cores</td><td>{driver.cores}</td>
-    </tr>
-    <tr>
-      <td>Memory</td><td>{driver.mem}</td>
-    </tr>
-    <tr>
-      <td>Submitted</td><td>{driver.submissionDate}</td>
-    </tr>
-    <tr>
-      <td>Supervise</td><td>{driver.supervise}</td>
-    </tr>
-  }
-
-  private def retryRow(retryState: Option[MesosClusterRetryState]): Seq[Node] = {
-    retryState.map { state =>
-      <tr>
-        <td>
-          {state.lastFailureStatus}
-        </td>
-        <td>
-          {state.nextRetry}
-        </td>
-        <td>
-          {state.retries}
-        </td>
-      </tr>
-    }.getOrElse(Seq[Node]())
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala
deleted file mode 100644
index 13ba7d3..0000000
--- a/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala
+++ /dev/null
@@ -1,136 +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.deploy.mesos.ui
-
-import javax.servlet.http.HttpServletRequest
-
-import scala.xml.Node
-
-import org.apache.mesos.Protos.TaskStatus
-
-import org.apache.spark.deploy.mesos.config._
-import org.apache.spark.deploy.mesos.MesosDriverDescription
-import org.apache.spark.scheduler.cluster.mesos.MesosClusterSubmissionState
-import org.apache.spark.ui.{UIUtils, WebUIPage}
-
-private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage("") {
-  private val historyServerURL = parent.conf.get(HISTORY_SERVER_URL)
-
-  def render(request: HttpServletRequest): Seq[Node] = {
-    val state = parent.scheduler.getSchedulerState()
-
-    val driverHeader = Seq("Driver ID")
-    val historyHeader = historyServerURL.map(url => Seq("History")).getOrElse(Nil)
-    val submissionHeader = Seq("Submit Date", "Main Class", "Driver Resources")
-
-    val queuedHeaders = driverHeader ++ submissionHeader
-    val driverHeaders = driverHeader ++ historyHeader ++ submissionHeader ++
-      Seq("Start Date", "Mesos Slave ID", "State")
-    val retryHeaders = Seq("Driver ID", "Submit Date", "Description") ++
-      Seq("Last Failed Status", "Next Retry Time", "Attempt Count")
-    val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers)
-    val launchedTable = UIUtils.listingTable(driverHeaders, driverRow, state.launchedDrivers)
-    val finishedTable = UIUtils.listingTable(driverHeaders, driverRow, state.finishedDrivers)
-    val retryTable = UIUtils.listingTable(retryHeaders, retryRow, state.pendingRetryDrivers)
-    val content =
-      <p>Mesos Framework ID: {state.frameworkId}</p>
-      <div class="row-fluid">
-        <div class="span12">
-          <h4>Queued Drivers:</h4>
-          {queuedTable}
-          <h4>Launched Drivers:</h4>
-          {launchedTable}
-          <h4>Finished Drivers:</h4>
-          {finishedTable}
-          <h4>Supervise drivers waiting for retry:</h4>
-          {retryTable}
-        </div>
-      </div>;
-    UIUtils.basicSparkPage(content, "Spark Drivers for Mesos cluster")
-  }
-
-  private def queuedRow(submission: MesosDriverDescription): Seq[Node] = {
-    val id = submission.submissionId
-    <tr>
-      <td><a href={s"driver?id=$id"}>{id}</a></td>
-      <td>{submission.submissionDate}</td>
-      <td>{submission.command.mainClass}</td>
-      <td>cpus: {submission.cores}, mem: {submission.mem}</td>
-    </tr>
-  }
-
-  private def driverRow(state: MesosClusterSubmissionState): Seq[Node] = {
-    val id = state.driverDescription.submissionId
-
-    val historyCol = if (historyServerURL.isDefined) {
-      <td>
-        <a href={s"${historyServerURL.get}/history/${state.frameworkId}"}>
-          {state.frameworkId}
-        </a>
-      </td>
-    } else Nil
-
-    <tr>
-      <td><a href={s"driver?id=$id"}>{id}</a></td>
-      {historyCol}
-      <td>{state.driverDescription.submissionDate}</td>
-      <td>{state.driverDescription.command.mainClass}</td>
-      <td>cpus: {state.driverDescription.cores}, mem: {state.driverDescription.mem}</td>
-      <td>{state.startDate}</td>
-      <td>{state.slaveId.getValue}</td>
-      <td>{stateString(state.mesosTaskStatus)}</td>
-    </tr>
-  }
-
-  private def retryRow(submission: MesosDriverDescription): Seq[Node] = {
-    val id = submission.submissionId
-    <tr>
-      <td><a href={s"driver?id=$id"}>{id}</a></td>
-      <td>{submission.submissionDate}</td>
-      <td>{submission.command.mainClass}</td>
-      <td>{submission.retryState.get.lastFailureStatus}</td>
-      <td>{submission.retryState.get.nextRetry}</td>
-      <td>{submission.retryState.get.retries}</td>
-    </tr>
-  }
-
-  private def stateString(status: Option[TaskStatus]): String = {
-    if (status.isEmpty) {
-      return ""
-    }
-    val sb = new StringBuilder
-    val s = status.get
-    sb.append(s"State: ${s.getState}")
-    if (status.get.hasMessage) {
-      sb.append(s", Message: ${s.getMessage}")
-    }
-    if (status.get.hasHealthy) {
-      sb.append(s", Healthy: ${s.getHealthy}")
-    }
-    if (status.get.hasSource) {
-      sb.append(s", Source: ${s.getSource}")
-    }
-    if (status.get.hasReason) {
-      sb.append(s", Reason: ${s.getReason}")
-    }
-    if (status.get.hasTimestamp) {
-      sb.append(s", Time: ${s.getTimestamp}")
-    }
-    sb.toString()
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala
deleted file mode 100644
index 6049789..0000000
--- a/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala
+++ /dev/null
@@ -1,49 +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.deploy.mesos.ui
-
-import org.apache.spark.{SecurityManager, SparkConf}
-import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler
-import org.apache.spark.ui.{SparkUI, WebUI}
-import org.apache.spark.ui.JettyUtils._
-
-/**
- * UI that displays driver results from the [[org.apache.spark.deploy.mesos.MesosClusterDispatcher]]
- */
-private[spark] class MesosClusterUI(
-    securityManager: SecurityManager,
-    port: Int,
-    val conf: SparkConf,
-    dispatcherPublicAddress: String,
-    val scheduler: MesosClusterScheduler)
-  extends WebUI(securityManager, securityManager.getSSLOptions("mesos"), port, conf) {
-
-  initialize()
-
-  def activeWebUiUrl: String = "http://" + dispatcherPublicAddress + ":" + boundPort
-
-  override def initialize() {
-    attachPage(new MesosClusterPage(this))
-    attachPage(new DriverPage(this))
-    attachHandler(createStaticHandler(MesosClusterUI.STATIC_RESOURCE_DIR, "/static"))
-  }
-}
-
-private object MesosClusterUI {
-  val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
deleted file mode 100644
index ff60b88..0000000
--- a/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
+++ /dev/null
@@ -1,156 +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.deploy.rest.mesos
-
-import java.io.File
-import java.text.SimpleDateFormat
-import java.util.{Date, Locale}
-import java.util.concurrent.atomic.AtomicLong
-import javax.servlet.http.HttpServletResponse
-
-import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf}
-import org.apache.spark.deploy.Command
-import org.apache.spark.deploy.mesos.MesosDriverDescription
-import org.apache.spark.deploy.rest._
-import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler
-import org.apache.spark.util.Utils
-
-/**
- * A server that responds to requests submitted by the [[RestSubmissionClient]].
- * All requests are forwarded to
- * [[org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler]].
- * This is intended to be used in Mesos cluster mode only.
- * For more details about the REST submission please refer to [[RestSubmissionServer]] javadocs.
- */
-private[spark] class MesosRestServer(
-    host: String,
-    requestedPort: Int,
-    masterConf: SparkConf,
-    scheduler: MesosClusterScheduler)
-  extends RestSubmissionServer(host, requestedPort, masterConf) {
-
-  protected override val submitRequestServlet =
-    new MesosSubmitRequestServlet(scheduler, masterConf)
-  protected override val killRequestServlet =
-    new MesosKillRequestServlet(scheduler, masterConf)
-  protected override val statusRequestServlet =
-    new MesosStatusRequestServlet(scheduler, masterConf)
-}
-
-private[mesos] class MesosSubmitRequestServlet(
-    scheduler: MesosClusterScheduler,
-    conf: SparkConf)
-  extends SubmitRequestServlet {
-
-  private val DEFAULT_SUPERVISE = false
-  private val DEFAULT_MEMORY = Utils.DEFAULT_DRIVER_MEM_MB // mb
-  private val DEFAULT_CORES = 1.0
-
-  private val nextDriverNumber = new AtomicLong(0)
-  // For application IDs
-  private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US)
-  private def newDriverId(submitDate: Date): String =
-    f"driver-${createDateFormat.format(submitDate)}-${nextDriverNumber.incrementAndGet()}%04d"
-
-  /**
-   * Build a driver description from the fields specified in the submit request.
-   *
-   * This involves constructing a command that launches a mesos framework for the job.
-   * This does not currently consider fields used by python applications since python
-   * is not supported in mesos cluster mode yet.
-   */
-  private def buildDriverDescription(request: CreateSubmissionRequest): MesosDriverDescription = {
-    // Required fields, including the main class because python is not yet supported
-    val appResource = Option(request.appResource).getOrElse {
-      throw new SubmitRestMissingFieldException("Application jar is missing.")
-    }
-    val mainClass = Option(request.mainClass).getOrElse {
-      throw new SubmitRestMissingFieldException("Main class is missing.")
-    }
-
-    // Optional fields
-    val sparkProperties = request.sparkProperties
-    val driverExtraJavaOptions = sparkProperties.get("spark.driver.extraJavaOptions")
-    val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath")
-    val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath")
-    val superviseDriver = sparkProperties.get("spark.driver.supervise")
-    val driverMemory = sparkProperties.get("spark.driver.memory")
-    val driverCores = sparkProperties.get("spark.driver.cores")
-    val appArgs = request.appArgs
-    val environmentVariables = request.environmentVariables
-    val name = request.sparkProperties.getOrElse("spark.app.name", mainClass)
-
-    // Construct driver description
-    val conf = new SparkConf(false).setAll(sparkProperties)
-    val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator))
-    val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator))
-    val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty)
-    val sparkJavaOpts = Utils.sparkJavaOpts(conf)
-    val javaOpts = sparkJavaOpts ++ extraJavaOpts
-    val command = new Command(
-      mainClass, appArgs, environmentVariables, extraClassPath, extraLibraryPath, javaOpts)
-    val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE)
-    val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY)
-    val actualDriverCores = driverCores.map(_.toDouble).getOrElse(DEFAULT_CORES)
-    val submitDate = new Date()
-    val submissionId = newDriverId(submitDate)
-
-    new MesosDriverDescription(
-      name, appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver,
-      command, request.sparkProperties, submissionId, submitDate)
-  }
-
-  protected override def handleSubmit(
-      requestMessageJson: String,
-      requestMessage: SubmitRestProtocolMessage,
-      responseServlet: HttpServletResponse): SubmitRestProtocolResponse = {
-    requestMessage match {
-      case submitRequest: CreateSubmissionRequest =>
-        val driverDescription = buildDriverDescription(submitRequest)
-        val s = scheduler.submitDriver(driverDescription)
-        s.serverSparkVersion = sparkVersion
-        val unknownFields = findUnknownFields(requestMessageJson, requestMessage)
-        if (unknownFields.nonEmpty) {
-          // If there are fields that the server does not know about, warn the client
-          s.unknownFields = unknownFields
-        }
-        s
-      case unexpected =>
-        responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST)
-        handleError(s"Received message of unexpected type ${unexpected.messageType}.")
-    }
-  }
-}
-
-private[mesos] class MesosKillRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf)
-  extends KillRequestServlet {
-  protected override def handleKill(submissionId: String): KillSubmissionResponse = {
-    val k = scheduler.killDriver(submissionId)
-    k.serverSparkVersion = sparkVersion
-    k
-  }
-}
-
-private[mesos] class MesosStatusRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf)
-  extends StatusRequestServlet {
-  protected override def handleStatus(submissionId: String): SubmissionStatusResponse = {
-    val d = scheduler.getDriverStatus(submissionId)
-    d.serverSparkVersion = sparkVersion
-    d
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
deleted file mode 100644
index ee9149c..0000000
--- a/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
+++ /dev/null
@@ -1,131 +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.executor
-
-import java.nio.ByteBuffer
-
-import scala.collection.JavaConverters._
-
-import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver}
-import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _}
-import org.apache.mesos.protobuf.ByteString
-
-import org.apache.spark.{SparkConf, SparkEnv, TaskState}
-import org.apache.spark.TaskState
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.internal.Logging
-import org.apache.spark.scheduler.cluster.mesos.{MesosSchedulerUtils, MesosTaskLaunchData}
-import org.apache.spark.util.Utils
-
-private[spark] class MesosExecutorBackend
-  extends MesosExecutor
-  with MesosSchedulerUtils // TODO: fix
-  with ExecutorBackend
-  with Logging {
-
-  var executor: Executor = null
-  var driver: ExecutorDriver = null
-
-  override def statusUpdate(taskId: Long, state: TaskState.TaskState, data: ByteBuffer) {
-    val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build()
-    driver.sendStatusUpdate(MesosTaskStatus.newBuilder()
-      .setTaskId(mesosTaskId)
-      .setState(taskStateToMesos(state))
-      .setData(ByteString.copyFrom(data))
-      .build())
-  }
-
-  override def registered(
-      driver: ExecutorDriver,
-      executorInfo: ExecutorInfo,
-      frameworkInfo: FrameworkInfo,
-      slaveInfo: SlaveInfo) {
-
-    // Get num cores for this task from ExecutorInfo, created in MesosSchedulerBackend.
-    val cpusPerTask = executorInfo.getResourcesList.asScala
-      .find(_.getName == "cpus")
-      .map(_.getScalar.getValue.toInt)
-      .getOrElse(0)
-    val executorId = executorInfo.getExecutorId.getValue
-
-    logInfo(s"Registered with Mesos as executor ID $executorId with $cpusPerTask cpus")
-    this.driver = driver
-    // Set a context class loader to be picked up by the serializer. Without this call
-    // the serializer would default to the null class loader, and fail to find Spark classes
-    // See SPARK-10986.
-    Thread.currentThread().setContextClassLoader(this.getClass.getClassLoader)
-
-    val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) ++
-      Seq[(String, String)](("spark.app.id", frameworkInfo.getId.getValue))
-    val conf = new SparkConf(loadDefaults = true).setAll(properties)
-    val port = conf.getInt("spark.executor.port", 0)
-    val env = SparkEnv.createExecutorEnv(
-      conf, executorId, slaveInfo.getHostname, port, cpusPerTask, None, isLocal = false)
-
-    executor = new Executor(
-      executorId,
-      slaveInfo.getHostname,
-      env)
-  }
-
-  override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) {
-    val taskId = taskInfo.getTaskId.getValue.toLong
-    val taskData = MesosTaskLaunchData.fromByteString(taskInfo.getData)
-    if (executor == null) {
-      logError("Received launchTask but executor was null")
-    } else {
-      SparkHadoopUtil.get.runAsSparkUser { () =>
-        executor.launchTask(this, taskId = taskId, attemptNumber = taskData.attemptNumber,
-          taskInfo.getName, taskData.serializedTask)
-      }
-    }
-  }
-
-  override def error(d: ExecutorDriver, message: String) {
-    logError("Error from Mesos: " + message)
-  }
-
-  override def killTask(d: ExecutorDriver, t: TaskID) {
-    if (executor == null) {
-      logError("Received KillTask but executor was null")
-    } else {
-      // TODO: Determine the 'interruptOnCancel' property set for the given job.
-      executor.killTask(t.getValue.toLong, interruptThread = false)
-    }
-  }
-
-  override def reregistered(d: ExecutorDriver, p2: SlaveInfo) {}
-
-  override def disconnected(d: ExecutorDriver) {}
-
-  override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]) {}
-
-  override def shutdown(d: ExecutorDriver) {}
-}
-
-/**
- * Entry point for Mesos executor.
- */
-private[spark] object MesosExecutorBackend extends Logging {
-  def main(args: Array[String]) {
-    Utils.initDaemon(log)
-    // Create a new Executor and start it running
-    val runner = new MesosExecutorBackend()
-    new MesosExecutorDriver(runner).run()
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala
deleted file mode 100644
index ed29b34..0000000
--- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala
+++ /dev/null
@@ -1,64 +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.mesos
-
-import org.apache.spark.{SparkContext, SparkException}
-import org.apache.spark.internal.config._
-import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
-
-/**
- * Cluster Manager for creation of Yarn scheduler and backend
- */
-private[spark] class MesosClusterManager extends ExternalClusterManager {
-  private val MESOS_REGEX = """mesos://(.*)""".r
-
-  override def canCreate(masterURL: String): Boolean = {
-    masterURL.startsWith("mesos")
-  }
-
-  override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = {
-    new TaskSchedulerImpl(sc)
-  }
-
-  override def createSchedulerBackend(sc: SparkContext,
-      masterURL: String,
-      scheduler: TaskScheduler): SchedulerBackend = {
-    require(!sc.conf.get(IO_ENCRYPTION_ENABLED),
-      "I/O encryption is currently not supported in Mesos.")
-
-    val mesosUrl = MESOS_REGEX.findFirstMatchIn(masterURL).get.group(1)
-    val coarse = sc.conf.getBoolean("spark.mesos.coarse", defaultValue = true)
-    if (coarse) {
-      new MesosCoarseGrainedSchedulerBackend(
-        scheduler.asInstanceOf[TaskSchedulerImpl],
-        sc,
-        mesosUrl,
-        sc.env.securityManager)
-    } else {
-      new MesosFineGrainedSchedulerBackend(
-        scheduler.asInstanceOf[TaskSchedulerImpl],
-        sc,
-        mesosUrl)
-    }
-  }
-
-  override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = {
-    scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend)
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala
deleted file mode 100644
index 61ab3e8..0000000
--- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala
+++ /dev/null
@@ -1,134 +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.mesos
-
-import scala.collection.JavaConverters._
-
-import org.apache.curator.framework.CuratorFramework
-import org.apache.zookeeper.CreateMode
-import org.apache.zookeeper.KeeperException.NoNodeException
-
-import org.apache.spark.SparkConf
-import org.apache.spark.deploy.SparkCuratorUtil
-import org.apache.spark.internal.Logging
-import org.apache.spark.util.Utils
-
-/**
- * Persistence engine factory that is responsible for creating new persistence engines
- * to store Mesos cluster mode state.
- */
-private[spark] abstract class MesosClusterPersistenceEngineFactory(conf: SparkConf) {
-  def createEngine(path: String): MesosClusterPersistenceEngine
-}
-
-/**
- * Mesos cluster persistence engine is responsible for persisting Mesos cluster mode
- * specific state, so that on failover all the state can be recovered and the scheduler
- * can resume managing the drivers.
- */
-private[spark] trait MesosClusterPersistenceEngine {
-  def persist(name: String, obj: Object): Unit
-  def expunge(name: String): Unit
-  def fetch[T](name: String): Option[T]
-  def fetchAll[T](): Iterable[T]
-}
-
-/**
- * Zookeeper backed persistence engine factory.
- * All Zk engines created from this factory shares the same Zookeeper client, so
- * all of them reuses the same connection pool.
- */
-private[spark] class ZookeeperMesosClusterPersistenceEngineFactory(conf: SparkConf)
-  extends MesosClusterPersistenceEngineFactory(conf) with Logging {
-
-  lazy val zk = SparkCuratorUtil.newClient(conf)
-
-  def createEngine(path: String): MesosClusterPersistenceEngine = {
-    new ZookeeperMesosClusterPersistenceEngine(path, zk, conf)
-  }
-}
-
-/**
- * Black hole persistence engine factory that creates black hole
- * persistence engines, which stores nothing.
- */
-private[spark] class BlackHoleMesosClusterPersistenceEngineFactory
-  extends MesosClusterPersistenceEngineFactory(null) {
-  def createEngine(path: String): MesosClusterPersistenceEngine = {
-    new BlackHoleMesosClusterPersistenceEngine
-  }
-}
-
-/**
- * Black hole persistence engine that stores nothing.
- */
-private[spark] class BlackHoleMesosClusterPersistenceEngine extends MesosClusterPersistenceEngine {
-  override def persist(name: String, obj: Object): Unit = {}
-  override def fetch[T](name: String): Option[T] = None
-  override def expunge(name: String): Unit = {}
-  override def fetchAll[T](): Iterable[T] = Iterable.empty[T]
-}
-
-/**
- * Zookeeper based Mesos cluster persistence engine, that stores cluster mode state
- * into Zookeeper. Each engine object is operating under one folder in Zookeeper, but
- * reuses a shared Zookeeper client.
- */
-private[spark] class ZookeeperMesosClusterPersistenceEngine(
-    baseDir: String,
-    zk: CuratorFramework,
-    conf: SparkConf)
-  extends MesosClusterPersistenceEngine with Logging {
-  private val WORKING_DIR =
-    conf.get("spark.deploy.zookeeper.dir", "/spark_mesos_dispatcher") + "/" + baseDir
-
-  SparkCuratorUtil.mkdir(zk, WORKING_DIR)
-
-  def path(name: String): String = {
-    WORKING_DIR + "/" + name
-  }
-
-  override def expunge(name: String): Unit = {
-    zk.delete().forPath(path(name))
-  }
-
-  override def persist(name: String, obj: Object): Unit = {
-    val serialized = Utils.serialize(obj)
-    val zkPath = path(name)
-    zk.create().withMode(CreateMode.PERSISTENT).forPath(zkPath, serialized)
-  }
-
-  override def fetch[T](name: String): Option[T] = {
-    val zkPath = path(name)
-
-    try {
-      val fileData = zk.getData().forPath(zkPath)
-      Some(Utils.deserialize[T](fileData))
-    } catch {
-      case e: NoNodeException => None
-      case e: Exception =>
-        logWarning("Exception while reading persisted file, deleting", e)
-        zk.delete().forPath(zkPath)
-        None
-    }
-  }
-
-  override def fetchAll[T](): Iterable[T] = {
-    zk.getChildren.forPath(WORKING_DIR).asScala.flatMap(fetch[T])
-  }
-}


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


[06/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
deleted file mode 100644
index 0378ef4..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ /dev/null
@@ -1,791 +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.deploy.yarn
-
-import java.io.{File, IOException}
-import java.lang.reflect.InvocationTargetException
-import java.net.{Socket, URI, URL}
-import java.util.concurrent.{TimeoutException, TimeUnit}
-
-import scala.collection.mutable.HashMap
-import scala.concurrent.Promise
-import scala.concurrent.duration.Duration
-import scala.util.control.NonFatal
-
-import org.apache.hadoop.fs.{FileSystem, Path}
-import org.apache.hadoop.yarn.api._
-import org.apache.hadoop.yarn.api.records._
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
-
-import org.apache.spark._
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.deploy.history.HistoryServer
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.deploy.yarn.security.{AMCredentialRenewer, ConfigurableCredentialManager}
-import org.apache.spark.internal.Logging
-import org.apache.spark.internal.config._
-import org.apache.spark.rpc._
-import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, YarnSchedulerBackend}
-import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
-import org.apache.spark.util._
-
-/**
- * Common application master functionality for Spark on Yarn.
- */
-private[spark] class ApplicationMaster(
-    args: ApplicationMasterArguments,
-    client: YarnRMClient)
-  extends Logging {
-
-  // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be
-  // optimal as more containers are available. Might need to handle this better.
-
-  private val sparkConf = new SparkConf()
-  private val yarnConf: YarnConfiguration = SparkHadoopUtil.get.newConfiguration(sparkConf)
-    .asInstanceOf[YarnConfiguration]
-  private val isClusterMode = args.userClass != null
-
-  // Default to twice the number of executors (twice the maximum number of executors if dynamic
-  // allocation is enabled), with a minimum of 3.
-
-  private val maxNumExecutorFailures = {
-    val effectiveNumExecutors =
-      if (Utils.isDynamicAllocationEnabled(sparkConf)) {
-        sparkConf.get(DYN_ALLOCATION_MAX_EXECUTORS)
-      } else {
-        sparkConf.get(EXECUTOR_INSTANCES).getOrElse(0)
-      }
-    // By default, effectiveNumExecutors is Int.MaxValue if dynamic allocation is enabled. We need
-    // avoid the integer overflow here.
-    val defaultMaxNumExecutorFailures = math.max(3,
-      if (effectiveNumExecutors > Int.MaxValue / 2) Int.MaxValue else (2 * effectiveNumExecutors))
-
-    sparkConf.get(MAX_EXECUTOR_FAILURES).getOrElse(defaultMaxNumExecutorFailures)
-  }
-
-  @volatile private var exitCode = 0
-  @volatile private var unregistered = false
-  @volatile private var finished = false
-  @volatile private var finalStatus = getDefaultFinalStatus
-  @volatile private var finalMsg: String = ""
-  @volatile private var userClassThread: Thread = _
-
-  @volatile private var reporterThread: Thread = _
-  @volatile private var allocator: YarnAllocator = _
-
-  // Lock for controlling the allocator (heartbeat) thread.
-  private val allocatorLock = new Object()
-
-  // Steady state heartbeat interval. We want to be reasonably responsive without causing too many
-  // requests to RM.
-  private val heartbeatInterval = {
-    // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses.
-    val expiryInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
-    math.max(0, math.min(expiryInterval / 2, sparkConf.get(RM_HEARTBEAT_INTERVAL)))
-  }
-
-  // Initial wait interval before allocator poll, to allow for quicker ramp up when executors are
-  // being requested.
-  private val initialAllocationInterval = math.min(heartbeatInterval,
-    sparkConf.get(INITIAL_HEARTBEAT_INTERVAL))
-
-  // Next wait interval before allocator poll.
-  private var nextAllocationInterval = initialAllocationInterval
-
-  private var rpcEnv: RpcEnv = null
-  private var amEndpoint: RpcEndpointRef = _
-
-  // In cluster mode, used to tell the AM when the user's SparkContext has been initialized.
-  private val sparkContextPromise = Promise[SparkContext]()
-
-  private var credentialRenewer: AMCredentialRenewer = _
-
-  // Load the list of localized files set by the client. This is used when launching executors,
-  // and is loaded here so that these configs don't pollute the Web UI's environment page in
-  // cluster mode.
-  private val localResources = {
-    logInfo("Preparing Local resources")
-    val resources = HashMap[String, LocalResource]()
-
-    def setupDistributedCache(
-        file: String,
-        rtype: LocalResourceType,
-        timestamp: String,
-        size: String,
-        vis: String): Unit = {
-      val uri = new URI(file)
-      val amJarRsrc = Records.newRecord(classOf[LocalResource])
-      amJarRsrc.setType(rtype)
-      amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis))
-      amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri))
-      amJarRsrc.setTimestamp(timestamp.toLong)
-      amJarRsrc.setSize(size.toLong)
-
-      val fileName = Option(uri.getFragment()).getOrElse(new Path(uri).getName())
-      resources(fileName) = amJarRsrc
-    }
-
-    val distFiles = sparkConf.get(CACHED_FILES)
-    val fileSizes = sparkConf.get(CACHED_FILES_SIZES)
-    val timeStamps = sparkConf.get(CACHED_FILES_TIMESTAMPS)
-    val visibilities = sparkConf.get(CACHED_FILES_VISIBILITIES)
-    val resTypes = sparkConf.get(CACHED_FILES_TYPES)
-
-    for (i <- 0 to distFiles.size - 1) {
-      val resType = LocalResourceType.valueOf(resTypes(i))
-      setupDistributedCache(distFiles(i), resType, timeStamps(i).toString, fileSizes(i).toString,
-      visibilities(i))
-    }
-
-    // Distribute the conf archive to executors.
-    sparkConf.get(CACHED_CONF_ARCHIVE).foreach { path =>
-      val uri = new URI(path)
-      val fs = FileSystem.get(uri, yarnConf)
-      val status = fs.getFileStatus(new Path(uri))
-      // SPARK-16080: Make sure to use the correct name for the destination when distributing the
-      // conf archive to executors.
-      val destUri = new URI(uri.getScheme(), uri.getRawSchemeSpecificPart(),
-        Client.LOCALIZED_CONF_DIR)
-      setupDistributedCache(destUri.toString(), LocalResourceType.ARCHIVE,
-        status.getModificationTime().toString, status.getLen.toString,
-        LocalResourceVisibility.PRIVATE.name())
-    }
-
-    // Clean up the configuration so it doesn't show up in the Web UI (since it's really noisy).
-    CACHE_CONFIGS.foreach { e =>
-      sparkConf.remove(e)
-      sys.props.remove(e.key)
-    }
-
-    resources.toMap
-  }
-
-  def getAttemptId(): ApplicationAttemptId = {
-    client.getAttemptId()
-  }
-
-  final def run(): Int = {
-    try {
-      val appAttemptId = client.getAttemptId()
-
-      var attemptID: Option[String] = None
-
-      if (isClusterMode) {
-        // Set the web ui port to be ephemeral for yarn so we don't conflict with
-        // other spark processes running on the same box
-        System.setProperty("spark.ui.port", "0")
-
-        // Set the master and deploy mode property to match the requested mode.
-        System.setProperty("spark.master", "yarn")
-        System.setProperty("spark.submit.deployMode", "cluster")
-
-        // Set this internal configuration if it is running on cluster mode, this
-        // configuration will be checked in SparkContext to avoid misuse of yarn cluster mode.
-        System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString())
-
-        attemptID = Option(appAttemptId.getAttemptId.toString)
-      }
-
-      new CallerContext(
-        "APPMASTER", sparkConf.get(APP_CALLER_CONTEXT),
-        Option(appAttemptId.getApplicationId.toString), attemptID).setCurrentContext()
-
-      logInfo("ApplicationAttemptId: " + appAttemptId)
-
-      val fs = FileSystem.get(yarnConf)
-
-      // This shutdown hook should run *after* the SparkContext is shut down.
-      val priority = ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY - 1
-      ShutdownHookManager.addShutdownHook(priority) { () =>
-        val maxAppAttempts = client.getMaxRegAttempts(sparkConf, yarnConf)
-        val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts
-
-        if (!finished) {
-          // The default state of ApplicationMaster is failed if it is invoked by shut down hook.
-          // This behavior is different compared to 1.x version.
-          // If user application is exited ahead of time by calling System.exit(N), here mark
-          // this application as failed with EXIT_EARLY. For a good shutdown, user shouldn't call
-          // System.exit(0) to terminate the application.
-          finish(finalStatus,
-            ApplicationMaster.EXIT_EARLY,
-            "Shutdown hook called before final status was reported.")
-        }
-
-        if (!unregistered) {
-          // we only want to unregister if we don't want the RM to retry
-          if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) {
-            unregister(finalStatus, finalMsg)
-            cleanupStagingDir(fs)
-          }
-        }
-      }
-
-      // Call this to force generation of secret so it gets populated into the
-      // Hadoop UGI. This has to happen before the startUserApplication which does a
-      // doAs in order for the credentials to be passed on to the executor containers.
-      val securityMgr = new SecurityManager(sparkConf)
-
-      // If the credentials file config is present, we must periodically renew tokens. So create
-      // a new AMDelegationTokenRenewer
-      if (sparkConf.contains(CREDENTIALS_FILE_PATH.key)) {
-        // If a principal and keytab have been set, use that to create new credentials for executors
-        // periodically
-        credentialRenewer =
-          new ConfigurableCredentialManager(sparkConf, yarnConf).credentialRenewer()
-        credentialRenewer.scheduleLoginFromKeytab()
-      }
-
-      if (isClusterMode) {
-        runDriver(securityMgr)
-      } else {
-        runExecutorLauncher(securityMgr)
-      }
-    } catch {
-      case e: Exception =>
-        // catch everything else if not specifically handled
-        logError("Uncaught exception: ", e)
-        finish(FinalApplicationStatus.FAILED,
-          ApplicationMaster.EXIT_UNCAUGHT_EXCEPTION,
-          "Uncaught exception: " + e)
-    }
-    exitCode
-  }
-
-  /**
-   * Set the default final application status for client mode to UNDEFINED to handle
-   * if YARN HA restarts the application so that it properly retries. Set the final
-   * status to SUCCEEDED in cluster mode to handle if the user calls System.exit
-   * from the application code.
-   */
-  final def getDefaultFinalStatus(): FinalApplicationStatus = {
-    if (isClusterMode) {
-      FinalApplicationStatus.FAILED
-    } else {
-      FinalApplicationStatus.UNDEFINED
-    }
-  }
-
-  /**
-   * unregister is used to completely unregister the application from the ResourceManager.
-   * This means the ResourceManager will not retry the application attempt on your behalf if
-   * a failure occurred.
-   */
-  final def unregister(status: FinalApplicationStatus, diagnostics: String = null): Unit = {
-    synchronized {
-      if (!unregistered) {
-        logInfo(s"Unregistering ApplicationMaster with $status" +
-          Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse(""))
-        unregistered = true
-        client.unregister(status, Option(diagnostics).getOrElse(""))
-      }
-    }
-  }
-
-  final def finish(status: FinalApplicationStatus, code: Int, msg: String = null): Unit = {
-    synchronized {
-      if (!finished) {
-        val inShutdown = ShutdownHookManager.inShutdown()
-        logInfo(s"Final app status: $status, exitCode: $code" +
-          Option(msg).map(msg => s", (reason: $msg)").getOrElse(""))
-        exitCode = code
-        finalStatus = status
-        finalMsg = msg
-        finished = true
-        if (!inShutdown && Thread.currentThread() != reporterThread && reporterThread != null) {
-          logDebug("shutting down reporter thread")
-          reporterThread.interrupt()
-        }
-        if (!inShutdown && Thread.currentThread() != userClassThread && userClassThread != null) {
-          logDebug("shutting down user thread")
-          userClassThread.interrupt()
-        }
-        if (!inShutdown && credentialRenewer != null) {
-          credentialRenewer.stop()
-          credentialRenewer = null
-        }
-      }
-    }
-  }
-
-  private def sparkContextInitialized(sc: SparkContext) = {
-    sparkContextPromise.success(sc)
-  }
-
-  private def registerAM(
-      _sparkConf: SparkConf,
-      _rpcEnv: RpcEnv,
-      driverRef: RpcEndpointRef,
-      uiAddress: String,
-      securityMgr: SecurityManager) = {
-    val appId = client.getAttemptId().getApplicationId().toString()
-    val attemptId = client.getAttemptId().getAttemptId().toString()
-    val historyAddress =
-      _sparkConf.get(HISTORY_SERVER_ADDRESS)
-        .map { text => SparkHadoopUtil.get.substituteHadoopVariables(text, yarnConf) }
-        .map { address => s"${address}${HistoryServer.UI_PATH_PREFIX}/${appId}/${attemptId}" }
-        .getOrElse("")
-
-    val driverUrl = RpcEndpointAddress(
-      _sparkConf.get("spark.driver.host"),
-      _sparkConf.get("spark.driver.port").toInt,
-      CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
-
-    // Before we initialize the allocator, let's log the information about how executors will
-    // be run up front, to avoid printing this out for every single executor being launched.
-    // Use placeholders for information that changes such as executor IDs.
-    logInfo {
-      val executorMemory = sparkConf.get(EXECUTOR_MEMORY).toInt
-      val executorCores = sparkConf.get(EXECUTOR_CORES)
-      val dummyRunner = new ExecutorRunnable(None, yarnConf, sparkConf, driverUrl, "<executorId>",
-        "<hostname>", executorMemory, executorCores, appId, securityMgr, localResources)
-      dummyRunner.launchContextDebugInfo()
-    }
-
-    allocator = client.register(driverUrl,
-      driverRef,
-      yarnConf,
-      _sparkConf,
-      uiAddress,
-      historyAddress,
-      securityMgr,
-      localResources)
-
-    allocator.allocateResources()
-    reporterThread = launchReporterThread()
-  }
-
-  /**
-   * Create an [[RpcEndpoint]] that communicates with the driver.
-   *
-   * In cluster mode, the AM and the driver belong to same process
-   * so the AMEndpoint need not monitor lifecycle of the driver.
-   *
-   * @return A reference to the driver's RPC endpoint.
-   */
-  private def runAMEndpoint(
-      host: String,
-      port: String,
-      isClusterMode: Boolean): RpcEndpointRef = {
-    val driverEndpoint = rpcEnv.setupEndpointRef(
-      RpcAddress(host, port.toInt),
-      YarnSchedulerBackend.ENDPOINT_NAME)
-    amEndpoint =
-      rpcEnv.setupEndpoint("YarnAM", new AMEndpoint(rpcEnv, driverEndpoint, isClusterMode))
-    driverEndpoint
-  }
-
-  private def runDriver(securityMgr: SecurityManager): Unit = {
-    addAmIpFilter()
-    userClassThread = startUserApplication()
-
-    // This a bit hacky, but we need to wait until the spark.driver.port property has
-    // been set by the Thread executing the user class.
-    logInfo("Waiting for spark context initialization...")
-    val totalWaitTime = sparkConf.get(AM_MAX_WAIT_TIME)
-    try {
-      val sc = ThreadUtils.awaitResult(sparkContextPromise.future,
-        Duration(totalWaitTime, TimeUnit.MILLISECONDS))
-      if (sc != null) {
-        rpcEnv = sc.env.rpcEnv
-        val driverRef = runAMEndpoint(
-          sc.getConf.get("spark.driver.host"),
-          sc.getConf.get("spark.driver.port"),
-          isClusterMode = true)
-        registerAM(sc.getConf, rpcEnv, driverRef, sc.ui.map(_.webUrl).getOrElse(""),
-          securityMgr)
-      } else {
-        // Sanity check; should never happen in normal operation, since sc should only be null
-        // if the user app did not create a SparkContext.
-        if (!finished) {
-          throw new IllegalStateException("SparkContext is null but app is still running!")
-        }
-      }
-      userClassThread.join()
-    } catch {
-      case e: SparkException if e.getCause().isInstanceOf[TimeoutException] =>
-        logError(
-          s"SparkContext did not initialize after waiting for $totalWaitTime ms. " +
-           "Please check earlier log output for errors. Failing the application.")
-        finish(FinalApplicationStatus.FAILED,
-          ApplicationMaster.EXIT_SC_NOT_INITED,
-          "Timed out waiting for SparkContext.")
-    }
-  }
-
-  private def runExecutorLauncher(securityMgr: SecurityManager): Unit = {
-    val port = sparkConf.get(AM_PORT)
-    rpcEnv = RpcEnv.create("sparkYarnAM", Utils.localHostName, port, sparkConf, securityMgr,
-      clientMode = true)
-    val driverRef = waitForSparkDriver()
-    addAmIpFilter()
-    registerAM(sparkConf, rpcEnv, driverRef, sparkConf.get("spark.driver.appUIAddress", ""),
-      securityMgr)
-
-    // In client mode the actor will stop the reporter thread.
-    reporterThread.join()
-  }
-
-  private def launchReporterThread(): Thread = {
-    // The number of failures in a row until Reporter thread give up
-    val reporterMaxFailures = sparkConf.get(MAX_REPORTER_THREAD_FAILURES)
-
-    val t = new Thread {
-      override def run() {
-        var failureCount = 0
-        while (!finished) {
-          try {
-            if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) {
-              finish(FinalApplicationStatus.FAILED,
-                ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES,
-                s"Max number of executor failures ($maxNumExecutorFailures) reached")
-            } else {
-              logDebug("Sending progress")
-              allocator.allocateResources()
-            }
-            failureCount = 0
-          } catch {
-            case i: InterruptedException =>
-            case e: Throwable =>
-              failureCount += 1
-              // this exception was introduced in hadoop 2.4 and this code would not compile
-              // with earlier versions if we refer it directly.
-              if ("org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException" ==
-                e.getClass().getName()) {
-                logError("Exception from Reporter thread.", e)
-                finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_REPORTER_FAILURE,
-                  e.getMessage)
-              } else if (!NonFatal(e) || failureCount >= reporterMaxFailures) {
-                finish(FinalApplicationStatus.FAILED,
-                  ApplicationMaster.EXIT_REPORTER_FAILURE, "Exception was thrown " +
-                    s"$failureCount time(s) from Reporter thread.")
-              } else {
-                logWarning(s"Reporter thread fails $failureCount time(s) in a row.", e)
-              }
-          }
-          try {
-            val numPendingAllocate = allocator.getPendingAllocate.size
-            var sleepStart = 0L
-            var sleepInterval = 200L // ms
-            allocatorLock.synchronized {
-              sleepInterval =
-                if (numPendingAllocate > 0 || allocator.getNumPendingLossReasonRequests > 0) {
-                  val currentAllocationInterval =
-                    math.min(heartbeatInterval, nextAllocationInterval)
-                  nextAllocationInterval = currentAllocationInterval * 2 // avoid overflow
-                  currentAllocationInterval
-                } else {
-                  nextAllocationInterval = initialAllocationInterval
-                  heartbeatInterval
-                }
-              sleepStart = System.currentTimeMillis()
-              allocatorLock.wait(sleepInterval)
-            }
-            val sleepDuration = System.currentTimeMillis() - sleepStart
-            if (sleepDuration < sleepInterval) {
-              // log when sleep is interrupted
-              logDebug(s"Number of pending allocations is $numPendingAllocate. " +
-                  s"Slept for $sleepDuration/$sleepInterval ms.")
-              // if sleep was less than the minimum interval, sleep for the rest of it
-              val toSleep = math.max(0, initialAllocationInterval - sleepDuration)
-              if (toSleep > 0) {
-                logDebug(s"Going back to sleep for $toSleep ms")
-                // use Thread.sleep instead of allocatorLock.wait. there is no need to be woken up
-                // by the methods that signal allocatorLock because this is just finishing the min
-                // sleep interval, which should happen even if this is signalled again.
-                Thread.sleep(toSleep)
-              }
-            } else {
-              logDebug(s"Number of pending allocations is $numPendingAllocate. " +
-                  s"Slept for $sleepDuration/$sleepInterval.")
-            }
-          } catch {
-            case e: InterruptedException =>
-          }
-        }
-      }
-    }
-    // setting to daemon status, though this is usually not a good idea.
-    t.setDaemon(true)
-    t.setName("Reporter")
-    t.start()
-    logInfo(s"Started progress reporter thread with (heartbeat : $heartbeatInterval, " +
-            s"initial allocation : $initialAllocationInterval) intervals")
-    t
-  }
-
-  /**
-   * Clean up the staging directory.
-   */
-  private def cleanupStagingDir(fs: FileSystem) {
-    var stagingDirPath: Path = null
-    try {
-      val preserveFiles = sparkConf.get(PRESERVE_STAGING_FILES)
-      if (!preserveFiles) {
-        stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR"))
-        if (stagingDirPath == null) {
-          logError("Staging directory is null")
-          return
-        }
-        logInfo("Deleting staging directory " + stagingDirPath)
-        fs.delete(stagingDirPath, true)
-      }
-    } catch {
-      case ioe: IOException =>
-        logError("Failed to cleanup staging dir " + stagingDirPath, ioe)
-    }
-  }
-
-  private def waitForSparkDriver(): RpcEndpointRef = {
-    logInfo("Waiting for Spark driver to be reachable.")
-    var driverUp = false
-    val hostport = args.userArgs(0)
-    val (driverHost, driverPort) = Utils.parseHostPort(hostport)
-
-    // Spark driver should already be up since it launched us, but we don't want to
-    // wait forever, so wait 100 seconds max to match the cluster mode setting.
-    val totalWaitTimeMs = sparkConf.get(AM_MAX_WAIT_TIME)
-    val deadline = System.currentTimeMillis + totalWaitTimeMs
-
-    while (!driverUp && !finished && System.currentTimeMillis < deadline) {
-      try {
-        val socket = new Socket(driverHost, driverPort)
-        socket.close()
-        logInfo("Driver now available: %s:%s".format(driverHost, driverPort))
-        driverUp = true
-      } catch {
-        case e: Exception =>
-          logError("Failed to connect to driver at %s:%s, retrying ...".
-            format(driverHost, driverPort))
-          Thread.sleep(100L)
-      }
-    }
-
-    if (!driverUp) {
-      throw new SparkException("Failed to connect to driver!")
-    }
-
-    sparkConf.set("spark.driver.host", driverHost)
-    sparkConf.set("spark.driver.port", driverPort.toString)
-
-    runAMEndpoint(driverHost, driverPort.toString, isClusterMode = false)
-  }
-
-  /** Add the Yarn IP filter that is required for properly securing the UI. */
-  private def addAmIpFilter() = {
-    val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV)
-    val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter"
-    val params = client.getAmIpFilterParams(yarnConf, proxyBase)
-    if (isClusterMode) {
-      System.setProperty("spark.ui.filters", amFilter)
-      params.foreach { case (k, v) => System.setProperty(s"spark.$amFilter.param.$k", v) }
-    } else {
-      amEndpoint.send(AddWebUIFilter(amFilter, params.toMap, proxyBase))
-    }
-  }
-
-  /**
-   * Start the user class, which contains the spark driver, in a separate Thread.
-   * If the main routine exits cleanly or exits with System.exit(N) for any N
-   * we assume it was successful, for all other cases we assume failure.
-   *
-   * Returns the user thread that was started.
-   */
-  private def startUserApplication(): Thread = {
-    logInfo("Starting the user application in a separate Thread")
-
-    val classpath = Client.getUserClasspath(sparkConf)
-    val urls = classpath.map { entry =>
-      new URL("file:" + new File(entry.getPath()).getAbsolutePath())
-    }
-    val userClassLoader =
-      if (Client.isUserClassPathFirst(sparkConf, isDriver = true)) {
-        new ChildFirstURLClassLoader(urls, Utils.getContextOrSparkClassLoader)
-      } else {
-        new MutableURLClassLoader(urls, Utils.getContextOrSparkClassLoader)
-      }
-
-    var userArgs = args.userArgs
-    if (args.primaryPyFile != null && args.primaryPyFile.endsWith(".py")) {
-      // When running pyspark, the app is run using PythonRunner. The second argument is the list
-      // of files to add to PYTHONPATH, which Client.scala already handles, so it's empty.
-      userArgs = Seq(args.primaryPyFile, "") ++ userArgs
-    }
-    if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) {
-      // TODO(davies): add R dependencies here
-    }
-    val mainMethod = userClassLoader.loadClass(args.userClass)
-      .getMethod("main", classOf[Array[String]])
-
-    val userThread = new Thread {
-      override def run() {
-        try {
-          mainMethod.invoke(null, userArgs.toArray)
-          finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS)
-          logDebug("Done running users class")
-        } catch {
-          case e: InvocationTargetException =>
-            e.getCause match {
-              case _: InterruptedException =>
-                // Reporter thread can interrupt to stop user class
-              case SparkUserAppException(exitCode) =>
-                val msg = s"User application exited with status $exitCode"
-                logError(msg)
-                finish(FinalApplicationStatus.FAILED, exitCode, msg)
-              case cause: Throwable =>
-                logError("User class threw exception: " + cause, cause)
-                finish(FinalApplicationStatus.FAILED,
-                  ApplicationMaster.EXIT_EXCEPTION_USER_CLASS,
-                  "User class threw exception: " + cause)
-            }
-            sparkContextPromise.tryFailure(e.getCause())
-        } finally {
-          // Notify the thread waiting for the SparkContext, in case the application did not
-          // instantiate one. This will do nothing when the user code instantiates a SparkContext
-          // (with the correct master), or when the user code throws an exception (due to the
-          // tryFailure above).
-          sparkContextPromise.trySuccess(null)
-        }
-      }
-    }
-    userThread.setContextClassLoader(userClassLoader)
-    userThread.setName("Driver")
-    userThread.start()
-    userThread
-  }
-
-  private def resetAllocatorInterval(): Unit = allocatorLock.synchronized {
-    nextAllocationInterval = initialAllocationInterval
-    allocatorLock.notifyAll()
-  }
-
-  /**
-   * An [[RpcEndpoint]] that communicates with the driver's scheduler backend.
-   */
-  private class AMEndpoint(
-      override val rpcEnv: RpcEnv, driver: RpcEndpointRef, isClusterMode: Boolean)
-    extends RpcEndpoint with Logging {
-
-    override def onStart(): Unit = {
-      driver.send(RegisterClusterManager(self))
-    }
-
-    override def receive: PartialFunction[Any, Unit] = {
-      case x: AddWebUIFilter =>
-        logInfo(s"Add WebUI Filter. $x")
-        driver.send(x)
-    }
-
-    override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
-      case RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount) =>
-        Option(allocator) match {
-          case Some(a) =>
-            if (a.requestTotalExecutorsWithPreferredLocalities(requestedTotal,
-              localityAwareTasks, hostToLocalTaskCount)) {
-              resetAllocatorInterval()
-            }
-            context.reply(true)
-
-          case None =>
-            logWarning("Container allocator is not ready to request executors yet.")
-            context.reply(false)
-        }
-
-      case KillExecutors(executorIds) =>
-        logInfo(s"Driver requested to kill executor(s) ${executorIds.mkString(", ")}.")
-        Option(allocator) match {
-          case Some(a) => executorIds.foreach(a.killExecutor)
-          case None => logWarning("Container allocator is not ready to kill executors yet.")
-        }
-        context.reply(true)
-
-      case GetExecutorLossReason(eid) =>
-        Option(allocator) match {
-          case Some(a) =>
-            a.enqueueGetLossReasonRequest(eid, context)
-            resetAllocatorInterval()
-          case None =>
-            logWarning("Container allocator is not ready to find executor loss reasons yet.")
-        }
-    }
-
-    override def onDisconnected(remoteAddress: RpcAddress): Unit = {
-      // In cluster mode, do not rely on the disassociated event to exit
-      // This avoids potentially reporting incorrect exit codes if the driver fails
-      if (!isClusterMode) {
-        logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress")
-        finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS)
-      }
-    }
-  }
-
-}
-
-object ApplicationMaster extends Logging {
-
-  // exit codes for different causes, no reason behind the values
-  private val EXIT_SUCCESS = 0
-  private val EXIT_UNCAUGHT_EXCEPTION = 10
-  private val EXIT_MAX_EXECUTOR_FAILURES = 11
-  private val EXIT_REPORTER_FAILURE = 12
-  private val EXIT_SC_NOT_INITED = 13
-  private val EXIT_SECURITY = 14
-  private val EXIT_EXCEPTION_USER_CLASS = 15
-  private val EXIT_EARLY = 16
-
-  private var master: ApplicationMaster = _
-
-  def main(args: Array[String]): Unit = {
-    SignalUtils.registerLogger(log)
-    val amArgs = new ApplicationMasterArguments(args)
-
-    // Load the properties file with the Spark configuration and set entries as system properties,
-    // so that user code run inside the AM also has access to them.
-    // Note: we must do this before SparkHadoopUtil instantiated
-    if (amArgs.propertiesFile != null) {
-      Utils.getPropertiesFromFile(amArgs.propertiesFile).foreach { case (k, v) =>
-        sys.props(k) = v
-      }
-    }
-    SparkHadoopUtil.get.runAsSparkUser { () =>
-      master = new ApplicationMaster(amArgs, new YarnRMClient)
-      System.exit(master.run())
-    }
-  }
-
-  private[spark] def sparkContextInitialized(sc: SparkContext): Unit = {
-    master.sparkContextInitialized(sc)
-  }
-
-  private[spark] def getAttemptId(): ApplicationAttemptId = {
-    master.getAttemptId
-  }
-
-}
-
-/**
- * This object does not provide any special functionality. It exists so that it's easy to tell
- * apart the client-mode AM from the cluster-mode AM when using tools such as ps or jps.
- */
-object ExecutorLauncher {
-
-  def main(args: Array[String]): Unit = {
-    ApplicationMaster.main(args)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
deleted file mode 100644
index 5cdec87..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
+++ /dev/null
@@ -1,105 +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.deploy.yarn
-
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.spark.util.{IntParam, MemoryParam}
-
-class ApplicationMasterArguments(val args: Array[String]) {
-  var userJar: String = null
-  var userClass: String = null
-  var primaryPyFile: String = null
-  var primaryRFile: String = null
-  var userArgs: Seq[String] = Nil
-  var propertiesFile: String = null
-
-  parseArgs(args.toList)
-
-  private def parseArgs(inputArgs: List[String]): Unit = {
-    val userArgsBuffer = new ArrayBuffer[String]()
-
-    var args = inputArgs
-
-    while (!args.isEmpty) {
-      // --num-workers, --worker-memory, and --worker-cores are deprecated since 1.0,
-      // the properties with executor in their names are preferred.
-      args match {
-        case ("--jar") :: value :: tail =>
-          userJar = value
-          args = tail
-
-        case ("--class") :: value :: tail =>
-          userClass = value
-          args = tail
-
-        case ("--primary-py-file") :: value :: tail =>
-          primaryPyFile = value
-          args = tail
-
-        case ("--primary-r-file") :: value :: tail =>
-          primaryRFile = value
-          args = tail
-
-        case ("--arg") :: value :: tail =>
-          userArgsBuffer += value
-          args = tail
-
-        case ("--properties-file") :: value :: tail =>
-          propertiesFile = value
-          args = tail
-
-        case _ =>
-          printUsageAndExit(1, args)
-      }
-    }
-
-    if (primaryPyFile != null && primaryRFile != null) {
-      // scalastyle:off println
-      System.err.println("Cannot have primary-py-file and primary-r-file at the same time")
-      // scalastyle:on println
-      System.exit(-1)
-    }
-
-    userArgs = userArgsBuffer.toList
-  }
-
-  def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
-    // scalastyle:off println
-    if (unknownParam != null) {
-      System.err.println("Unknown/unsupported param " + unknownParam)
-    }
-    System.err.println("""
-      |Usage: org.apache.spark.deploy.yarn.ApplicationMaster [options]
-      |Options:
-      |  --jar JAR_PATH       Path to your application's JAR file
-      |  --class CLASS_NAME   Name of your application's main class
-      |  --primary-py-file    A main Python file
-      |  --primary-r-file     A main R file
-      |  --arg ARG            Argument to be passed to your application's main class.
-      |                       Multiple invocations are possible, each will be passed in order.
-      |  --properties-file FILE Path to a custom Spark properties file.
-      """.stripMargin)
-    // scalastyle:on println
-    System.exit(exitCode)
-  }
-}
-
-object ApplicationMasterArguments {
-  val DEFAULT_NUMBER_EXECUTORS = 2
-}


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


[11/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
new file mode 100644
index 0000000..be419ce
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -0,0 +1,1541 @@
+/*
+ * 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 java.io.{File, FileOutputStream, IOException, OutputStreamWriter}
+import java.net.{InetAddress, UnknownHostException, URI}
+import java.nio.ByteBuffer
+import java.nio.charset.StandardCharsets
+import java.util.{Properties, UUID}
+import java.util.zip.{ZipEntry, ZipOutputStream}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map}
+import scala.util.{Failure, Success, Try}
+import scala.util.control.NonFatal
+
+import com.google.common.base.Objects
+import com.google.common.io.Files
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs._
+import org.apache.hadoop.fs.permission.FsPermission
+import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.mapreduce.MRJobConfig
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+import org.apache.hadoop.util.StringUtils
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication}
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException
+import org.apache.hadoop.yarn.util.Records
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkException}
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils}
+import org.apache.spark.util.{CallerContext, Utils}
+
+private[spark] class Client(
+    val args: ClientArguments,
+    val hadoopConf: Configuration,
+    val sparkConf: SparkConf)
+  extends Logging {
+
+  import Client._
+  import YarnSparkHadoopUtil._
+
+  def this(clientArgs: ClientArguments, spConf: SparkConf) =
+    this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf)
+
+  private val yarnClient = YarnClient.createYarnClient
+  private val yarnConf = new YarnConfiguration(hadoopConf)
+
+  private val isClusterMode = sparkConf.get("spark.submit.deployMode", "client") == "cluster"
+
+  // AM related configurations
+  private val amMemory = if (isClusterMode) {
+    sparkConf.get(DRIVER_MEMORY).toInt
+  } else {
+    sparkConf.get(AM_MEMORY).toInt
+  }
+  private val amMemoryOverhead = {
+    val amMemoryOverheadEntry = if (isClusterMode) DRIVER_MEMORY_OVERHEAD else AM_MEMORY_OVERHEAD
+    sparkConf.get(amMemoryOverheadEntry).getOrElse(
+      math.max((MEMORY_OVERHEAD_FACTOR * amMemory).toLong, MEMORY_OVERHEAD_MIN)).toInt
+  }
+  private val amCores = if (isClusterMode) {
+    sparkConf.get(DRIVER_CORES)
+  } else {
+    sparkConf.get(AM_CORES)
+  }
+
+  // Executor related configurations
+  private val executorMemory = sparkConf.get(EXECUTOR_MEMORY)
+  private val executorMemoryOverhead = sparkConf.get(EXECUTOR_MEMORY_OVERHEAD).getOrElse(
+    math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toLong, MEMORY_OVERHEAD_MIN)).toInt
+
+  private val distCacheMgr = new ClientDistributedCacheManager()
+
+  private var loginFromKeytab = false
+  private var principal: String = null
+  private var keytab: String = null
+  private var credentials: Credentials = null
+
+  private val launcherBackend = new LauncherBackend() {
+    override def onStopRequest(): Unit = {
+      if (isClusterMode && appId != null) {
+        yarnClient.killApplication(appId)
+      } else {
+        setState(SparkAppHandle.State.KILLED)
+        stop()
+      }
+    }
+  }
+  private val fireAndForget = isClusterMode && !sparkConf.get(WAIT_FOR_APP_COMPLETION)
+
+  private var appId: ApplicationId = null
+
+  // The app staging dir based on the STAGING_DIR configuration if configured
+  // otherwise based on the users home directory.
+  private val appStagingBaseDir = sparkConf.get(STAGING_DIR).map { new Path(_) }
+    .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory())
+
+  private val credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
+
+  def reportLauncherState(state: SparkAppHandle.State): Unit = {
+    launcherBackend.setState(state)
+  }
+
+  def stop(): Unit = {
+    launcherBackend.close()
+    yarnClient.stop()
+    // Unset YARN mode system env variable, to allow switching between cluster types.
+    System.clearProperty("SPARK_YARN_MODE")
+  }
+
+  /**
+   * Submit an application running our ApplicationMaster to the ResourceManager.
+   *
+   * The stable Yarn API provides a convenience method (YarnClient#createApplication) for
+   * creating applications and setting up the application submission context. This was not
+   * available in the alpha API.
+   */
+  def submitApplication(): ApplicationId = {
+    var appId: ApplicationId = null
+    try {
+      launcherBackend.connect()
+      // Setup the credentials before doing anything else,
+      // so we have don't have issues at any point.
+      setupCredentials()
+      yarnClient.init(yarnConf)
+      yarnClient.start()
+
+      logInfo("Requesting a new application from cluster with %d NodeManagers"
+        .format(yarnClient.getYarnClusterMetrics.getNumNodeManagers))
+
+      // Get a new application from our RM
+      val newApp = yarnClient.createApplication()
+      val newAppResponse = newApp.getNewApplicationResponse()
+      appId = newAppResponse.getApplicationId()
+      reportLauncherState(SparkAppHandle.State.SUBMITTED)
+      launcherBackend.setAppId(appId.toString)
+
+      new CallerContext("CLIENT", sparkConf.get(APP_CALLER_CONTEXT),
+        Option(appId.toString)).setCurrentContext()
+
+      // Verify whether the cluster has enough resources for our AM
+      verifyClusterResources(newAppResponse)
+
+      // Set up the appropriate contexts to launch our AM
+      val containerContext = createContainerLaunchContext(newAppResponse)
+      val appContext = createApplicationSubmissionContext(newApp, containerContext)
+
+      // Finally, submit and monitor the application
+      logInfo(s"Submitting application $appId to ResourceManager")
+      yarnClient.submitApplication(appContext)
+      appId
+    } catch {
+      case e: Throwable =>
+        if (appId != null) {
+          cleanupStagingDir(appId)
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Cleanup application staging directory.
+   */
+  private def cleanupStagingDir(appId: ApplicationId): Unit = {
+    val stagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId))
+    try {
+      val preserveFiles = sparkConf.get(PRESERVE_STAGING_FILES)
+      val fs = stagingDirPath.getFileSystem(hadoopConf)
+      if (!preserveFiles && fs.delete(stagingDirPath, true)) {
+        logInfo(s"Deleted staging directory $stagingDirPath")
+      }
+    } catch {
+      case ioe: IOException =>
+        logWarning("Failed to cleanup staging dir " + stagingDirPath, ioe)
+    }
+  }
+
+  /**
+   * Set up the context for submitting our ApplicationMaster.
+   * This uses the YarnClientApplication not available in the Yarn alpha API.
+   */
+  def createApplicationSubmissionContext(
+      newApp: YarnClientApplication,
+      containerContext: ContainerLaunchContext): ApplicationSubmissionContext = {
+    val appContext = newApp.getApplicationSubmissionContext
+    appContext.setApplicationName(sparkConf.get("spark.app.name", "Spark"))
+    appContext.setQueue(sparkConf.get(QUEUE_NAME))
+    appContext.setAMContainerSpec(containerContext)
+    appContext.setApplicationType("SPARK")
+
+    sparkConf.get(APPLICATION_TAGS).foreach { tags =>
+      try {
+        // The setApplicationTags method was only introduced in Hadoop 2.4+, so we need to use
+        // reflection to set it, printing a warning if a tag was specified but the YARN version
+        // doesn't support it.
+        val method = appContext.getClass().getMethod(
+          "setApplicationTags", classOf[java.util.Set[String]])
+        method.invoke(appContext, new java.util.HashSet[String](tags.asJava))
+      } catch {
+        case e: NoSuchMethodException =>
+          logWarning(s"Ignoring ${APPLICATION_TAGS.key} because this version of " +
+            "YARN does not support it")
+      }
+    }
+    sparkConf.get(MAX_APP_ATTEMPTS) match {
+      case Some(v) => appContext.setMaxAppAttempts(v)
+      case None => logDebug(s"${MAX_APP_ATTEMPTS.key} is not set. " +
+          "Cluster's default value will be used.")
+    }
+
+    sparkConf.get(AM_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS).foreach { interval =>
+      try {
+        val method = appContext.getClass().getMethod(
+          "setAttemptFailuresValidityInterval", classOf[Long])
+        method.invoke(appContext, interval: java.lang.Long)
+      } catch {
+        case e: NoSuchMethodException =>
+          logWarning(s"Ignoring ${AM_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS.key} because " +
+            "the version of YARN does not support it")
+      }
+    }
+
+    val capability = Records.newRecord(classOf[Resource])
+    capability.setMemory(amMemory + amMemoryOverhead)
+    capability.setVirtualCores(amCores)
+
+    sparkConf.get(AM_NODE_LABEL_EXPRESSION) match {
+      case Some(expr) =>
+        try {
+          val amRequest = Records.newRecord(classOf[ResourceRequest])
+          amRequest.setResourceName(ResourceRequest.ANY)
+          amRequest.setPriority(Priority.newInstance(0))
+          amRequest.setCapability(capability)
+          amRequest.setNumContainers(1)
+          val method = amRequest.getClass.getMethod("setNodeLabelExpression", classOf[String])
+          method.invoke(amRequest, expr)
+
+          val setResourceRequestMethod =
+            appContext.getClass.getMethod("setAMContainerResourceRequest", classOf[ResourceRequest])
+          setResourceRequestMethod.invoke(appContext, amRequest)
+        } catch {
+          case e: NoSuchMethodException =>
+            logWarning(s"Ignoring ${AM_NODE_LABEL_EXPRESSION.key} because the version " +
+              "of YARN does not support it")
+            appContext.setResource(capability)
+        }
+      case None =>
+        appContext.setResource(capability)
+    }
+
+    sparkConf.get(ROLLED_LOG_INCLUDE_PATTERN).foreach { includePattern =>
+      try {
+        val logAggregationContext = Records.newRecord(
+          Utils.classForName("org.apache.hadoop.yarn.api.records.LogAggregationContext"))
+          .asInstanceOf[Object]
+
+        val setRolledLogsIncludePatternMethod =
+          logAggregationContext.getClass.getMethod("setRolledLogsIncludePattern", classOf[String])
+        setRolledLogsIncludePatternMethod.invoke(logAggregationContext, includePattern)
+
+        sparkConf.get(ROLLED_LOG_EXCLUDE_PATTERN).foreach { excludePattern =>
+          val setRolledLogsExcludePatternMethod =
+            logAggregationContext.getClass.getMethod("setRolledLogsExcludePattern", classOf[String])
+          setRolledLogsExcludePatternMethod.invoke(logAggregationContext, excludePattern)
+        }
+
+        val setLogAggregationContextMethod =
+          appContext.getClass.getMethod("setLogAggregationContext",
+            Utils.classForName("org.apache.hadoop.yarn.api.records.LogAggregationContext"))
+        setLogAggregationContextMethod.invoke(appContext, logAggregationContext)
+      } catch {
+        case NonFatal(e) =>
+          logWarning(s"Ignoring ${ROLLED_LOG_INCLUDE_PATTERN.key} because the version of YARN " +
+            s"does not support it", e)
+      }
+    }
+
+    appContext
+  }
+
+  /** Set up security tokens for launching our ApplicationMaster container. */
+  private def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = {
+    val dob = new DataOutputBuffer
+    credentials.writeTokenStorageToStream(dob)
+    amContainer.setTokens(ByteBuffer.wrap(dob.getData))
+  }
+
+  /** Get the application report from the ResourceManager for an application we have submitted. */
+  def getApplicationReport(appId: ApplicationId): ApplicationReport =
+    yarnClient.getApplicationReport(appId)
+
+  /**
+   * Return the security token used by this client to communicate with the ApplicationMaster.
+   * If no security is enabled, the token returned by the report is null.
+   */
+  private def getClientToken(report: ApplicationReport): String =
+    Option(report.getClientToAMToken).map(_.toString).getOrElse("")
+
+  /**
+   * Fail fast if we have requested more resources per container than is available in the cluster.
+   */
+  private def verifyClusterResources(newAppResponse: GetNewApplicationResponse): Unit = {
+    val maxMem = newAppResponse.getMaximumResourceCapability().getMemory()
+    logInfo("Verifying our application has not requested more than the maximum " +
+      s"memory capability of the cluster ($maxMem MB per container)")
+    val executorMem = executorMemory + executorMemoryOverhead
+    if (executorMem > maxMem) {
+      throw new IllegalArgumentException(s"Required executor memory ($executorMemory" +
+        s"+$executorMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster! " +
+        "Please check the values of 'yarn.scheduler.maximum-allocation-mb' and/or " +
+        "'yarn.nodemanager.resource.memory-mb'.")
+    }
+    val amMem = amMemory + amMemoryOverhead
+    if (amMem > maxMem) {
+      throw new IllegalArgumentException(s"Required AM memory ($amMemory" +
+        s"+$amMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster! " +
+        "Please increase the value of 'yarn.scheduler.maximum-allocation-mb'.")
+    }
+    logInfo("Will allocate AM container, with %d MB memory including %d MB overhead".format(
+      amMem,
+      amMemoryOverhead))
+
+    // We could add checks to make sure the entire cluster has enough resources but that involves
+    // getting all the node reports and computing ourselves.
+  }
+
+  /**
+   * Copy the given file to a remote file system (e.g. HDFS) if needed.
+   * The file is only copied if the source and destination file systems are different. This is used
+   * for preparing resources for launching the ApplicationMaster container. Exposed for testing.
+   */
+  private[yarn] def copyFileToRemote(
+      destDir: Path,
+      srcPath: Path,
+      replication: Short,
+      force: Boolean = false,
+      destName: Option[String] = None): Path = {
+    val destFs = destDir.getFileSystem(hadoopConf)
+    val srcFs = srcPath.getFileSystem(hadoopConf)
+    var destPath = srcPath
+    if (force || !compareFs(srcFs, destFs)) {
+      destPath = new Path(destDir, destName.getOrElse(srcPath.getName()))
+      logInfo(s"Uploading resource $srcPath -> $destPath")
+      FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf)
+      destFs.setReplication(destPath, replication)
+      destFs.setPermission(destPath, new FsPermission(APP_FILE_PERMISSION))
+    } else {
+      logInfo(s"Source and destination file systems are the same. Not copying $srcPath")
+    }
+    // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
+    // version shows the specific version in the distributed cache configuration
+    val qualifiedDestPath = destFs.makeQualified(destPath)
+    val fc = FileContext.getFileContext(qualifiedDestPath.toUri(), hadoopConf)
+    fc.resolvePath(qualifiedDestPath)
+  }
+
+  /**
+   * Upload any resources to the distributed cache if needed. If a resource is intended to be
+   * consumed locally, set up the appropriate config for downstream code to handle it properly.
+   * This is used for setting up a container launch context for our ApplicationMaster.
+   * Exposed for testing.
+   */
+  def prepareLocalResources(
+      destDir: Path,
+      pySparkArchives: Seq[String]): HashMap[String, LocalResource] = {
+    logInfo("Preparing resources for our AM container")
+    // Upload Spark and the application JAR to the remote file system if necessary,
+    // and add them as local resources to the application master.
+    val fs = destDir.getFileSystem(hadoopConf)
+
+    // Merge credentials obtained from registered providers
+    val nearestTimeOfNextRenewal = credentialManager.obtainCredentials(hadoopConf, credentials)
+
+    if (credentials != null) {
+      logDebug(YarnSparkHadoopUtil.get.dumpTokens(credentials).mkString("\n"))
+    }
+
+    // If we use principal and keytab to login, also credentials can be renewed some time
+    // after current time, we should pass the next renewal and updating time to credential
+    // renewer and updater.
+    if (loginFromKeytab && nearestTimeOfNextRenewal > System.currentTimeMillis() &&
+      nearestTimeOfNextRenewal != Long.MaxValue) {
+
+      // Valid renewal time is 75% of next renewal time, and the valid update time will be
+      // slightly later then renewal time (80% of next renewal time). This is to make sure
+      // credentials are renewed and updated before expired.
+      val currTime = System.currentTimeMillis()
+      val renewalTime = (nearestTimeOfNextRenewal - currTime) * 0.75 + currTime
+      val updateTime = (nearestTimeOfNextRenewal - currTime) * 0.8 + currTime
+
+      sparkConf.set(CREDENTIALS_RENEWAL_TIME, renewalTime.toLong)
+      sparkConf.set(CREDENTIALS_UPDATE_TIME, updateTime.toLong)
+    }
+
+    // Used to keep track of URIs added to the distributed cache. If the same URI is added
+    // multiple times, YARN will fail to launch containers for the app with an internal
+    // error.
+    val distributedUris = new HashSet[String]
+    // Used to keep track of URIs(files) added to the distribute cache have the same name. If
+    // same name but different path files are added multiple time, YARN will fail to launch
+    // containers for the app with an internal error.
+    val distributedNames = new HashSet[String]
+
+    val replication = sparkConf.get(STAGING_FILE_REPLICATION).map(_.toShort)
+      .getOrElse(fs.getDefaultReplication(destDir))
+    val localResources = HashMap[String, LocalResource]()
+    FileSystem.mkdirs(fs, destDir, new FsPermission(STAGING_DIR_PERMISSION))
+
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+
+    def addDistributedUri(uri: URI): Boolean = {
+      val uriStr = uri.toString()
+      val fileName = new File(uri.getPath).getName
+      if (distributedUris.contains(uriStr)) {
+        logWarning(s"Same path resource $uri added multiple times to distributed cache.")
+        false
+      } else if (distributedNames.contains(fileName)) {
+        logWarning(s"Same name resource $uri added multiple times to distributed cache")
+        false
+      } else {
+        distributedUris += uriStr
+        distributedNames += fileName
+        true
+      }
+    }
+
+    /**
+     * Distribute a file to the cluster.
+     *
+     * If the file's path is a "local:" URI, it's actually not distributed. Other files are copied
+     * to HDFS (if not already there) and added to the application's distributed cache.
+     *
+     * @param path URI of the file to distribute.
+     * @param resType Type of resource being distributed.
+     * @param destName Name of the file in the distributed cache.
+     * @param targetDir Subdirectory where to place the file.
+     * @param appMasterOnly Whether to distribute only to the AM.
+     * @return A 2-tuple. First item is whether the file is a "local:" URI. Second item is the
+     *         localized path for non-local paths, or the input `path` for local paths.
+     *         The localized path will be null if the URI has already been added to the cache.
+     */
+    def distribute(
+        path: String,
+        resType: LocalResourceType = LocalResourceType.FILE,
+        destName: Option[String] = None,
+        targetDir: Option[String] = None,
+        appMasterOnly: Boolean = false): (Boolean, String) = {
+      val trimmedPath = path.trim()
+      val localURI = Utils.resolveURI(trimmedPath)
+      if (localURI.getScheme != LOCAL_SCHEME) {
+        if (addDistributedUri(localURI)) {
+          val localPath = getQualifiedLocalPath(localURI, hadoopConf)
+          val linkname = targetDir.map(_ + "/").getOrElse("") +
+            destName.orElse(Option(localURI.getFragment())).getOrElse(localPath.getName())
+          val destPath = copyFileToRemote(destDir, localPath, replication)
+          val destFs = FileSystem.get(destPath.toUri(), hadoopConf)
+          distCacheMgr.addResource(
+            destFs, hadoopConf, destPath, localResources, resType, linkname, statCache,
+            appMasterOnly = appMasterOnly)
+          (false, linkname)
+        } else {
+          (false, null)
+        }
+      } else {
+        (true, trimmedPath)
+      }
+    }
+
+    // If we passed in a keytab, make sure we copy the keytab to the staging directory on
+    // HDFS, and setup the relevant environment vars, so the AM can login again.
+    if (loginFromKeytab) {
+      logInfo("To enable the AM to login from keytab, credentials are being copied over to the AM" +
+        " via the YARN Secure Distributed Cache.")
+      val (_, localizedPath) = distribute(keytab,
+        destName = sparkConf.get(KEYTAB),
+        appMasterOnly = true)
+      require(localizedPath != null, "Keytab file already distributed.")
+    }
+
+    /**
+     * Add Spark to the cache. There are two settings that control what files to add to the cache:
+     * - if a Spark archive is defined, use the archive. The archive is expected to contain
+     *   jar files at its root directory.
+     * - if a list of jars is provided, filter the non-local ones, resolve globs, and
+     *   add the found files to the cache.
+     *
+     * Note that the archive cannot be a "local" URI. If none of the above settings are found,
+     * then upload all files found in $SPARK_HOME/jars.
+     */
+    val sparkArchive = sparkConf.get(SPARK_ARCHIVE)
+    if (sparkArchive.isDefined) {
+      val archive = sparkArchive.get
+      require(!isLocalUri(archive), s"${SPARK_ARCHIVE.key} cannot be a local URI.")
+      distribute(Utils.resolveURI(archive).toString,
+        resType = LocalResourceType.ARCHIVE,
+        destName = Some(LOCALIZED_LIB_DIR))
+    } else {
+      sparkConf.get(SPARK_JARS) match {
+        case Some(jars) =>
+          // Break the list of jars to upload, and resolve globs.
+          val localJars = new ArrayBuffer[String]()
+          jars.foreach { jar =>
+            if (!isLocalUri(jar)) {
+              val path = getQualifiedLocalPath(Utils.resolveURI(jar), hadoopConf)
+              val pathFs = FileSystem.get(path.toUri(), hadoopConf)
+              pathFs.globStatus(path).filter(_.isFile()).foreach { entry =>
+                distribute(entry.getPath().toUri().toString(),
+                  targetDir = Some(LOCALIZED_LIB_DIR))
+              }
+            } else {
+              localJars += jar
+            }
+          }
+
+          // Propagate the local URIs to the containers using the configuration.
+          sparkConf.set(SPARK_JARS, localJars)
+
+        case None =>
+          // No configuration, so fall back to uploading local jar files.
+          logWarning(s"Neither ${SPARK_JARS.key} nor ${SPARK_ARCHIVE.key} is set, falling back " +
+            "to uploading libraries under SPARK_HOME.")
+          val jarsDir = new File(YarnCommandBuilderUtils.findJarsDir(
+            sparkConf.getenv("SPARK_HOME")))
+          val jarsArchive = File.createTempFile(LOCALIZED_LIB_DIR, ".zip",
+            new File(Utils.getLocalDir(sparkConf)))
+          val jarsStream = new ZipOutputStream(new FileOutputStream(jarsArchive))
+
+          try {
+            jarsStream.setLevel(0)
+            jarsDir.listFiles().foreach { f =>
+              if (f.isFile && f.getName.toLowerCase().endsWith(".jar") && f.canRead) {
+                jarsStream.putNextEntry(new ZipEntry(f.getName))
+                Files.copy(f, jarsStream)
+                jarsStream.closeEntry()
+              }
+            }
+          } finally {
+            jarsStream.close()
+          }
+
+          distribute(jarsArchive.toURI.getPath,
+            resType = LocalResourceType.ARCHIVE,
+            destName = Some(LOCALIZED_LIB_DIR))
+      }
+    }
+
+    /**
+     * Copy user jar to the distributed cache if their scheme is not "local".
+     * Otherwise, set the corresponding key in our SparkConf to handle it downstream.
+     */
+    Option(args.userJar).filter(_.trim.nonEmpty).foreach { jar =>
+      val (isLocal, localizedPath) = distribute(jar, destName = Some(APP_JAR_NAME))
+      if (isLocal) {
+        require(localizedPath != null, s"Path $jar already distributed")
+        // If the resource is intended for local use only, handle this downstream
+        // by setting the appropriate property
+        sparkConf.set(APP_JAR, localizedPath)
+      }
+    }
+
+    /**
+     * Do the same for any additional resources passed in through ClientArguments.
+     * Each resource category is represented by a 3-tuple of:
+     *   (1) comma separated list of resources in this category,
+     *   (2) resource type, and
+     *   (3) whether to add these resources to the classpath
+     */
+    val cachedSecondaryJarLinks = ListBuffer.empty[String]
+    List(
+      (sparkConf.get(JARS_TO_DISTRIBUTE), LocalResourceType.FILE, true),
+      (sparkConf.get(FILES_TO_DISTRIBUTE), LocalResourceType.FILE, false),
+      (sparkConf.get(ARCHIVES_TO_DISTRIBUTE), LocalResourceType.ARCHIVE, false)
+    ).foreach { case (flist, resType, addToClasspath) =>
+      flist.foreach { file =>
+        val (_, localizedPath) = distribute(file, resType = resType)
+        // If addToClassPath, we ignore adding jar multiple times to distitrbuted cache.
+        if (addToClasspath) {
+          if (localizedPath != null) {
+            cachedSecondaryJarLinks += localizedPath
+          }
+        } else {
+          if (localizedPath == null) {
+            throw new IllegalArgumentException(s"Attempt to add ($file) multiple times" +
+              " to the distributed cache.")
+          }
+        }
+      }
+    }
+    if (cachedSecondaryJarLinks.nonEmpty) {
+      sparkConf.set(SECONDARY_JARS, cachedSecondaryJarLinks)
+    }
+
+    if (isClusterMode && args.primaryPyFile != null) {
+      distribute(args.primaryPyFile, appMasterOnly = true)
+    }
+
+    pySparkArchives.foreach { f => distribute(f) }
+
+    // The python files list needs to be treated especially. All files that are not an
+    // archive need to be placed in a subdirectory that will be added to PYTHONPATH.
+    sparkConf.get(PY_FILES).foreach { f =>
+      val targetDir = if (f.endsWith(".py")) Some(LOCALIZED_PYTHON_DIR) else None
+      distribute(f, targetDir = targetDir)
+    }
+
+    // Update the configuration with all the distributed files, minus the conf archive. The
+    // conf archive will be handled by the AM differently so that we avoid having to send
+    // this configuration by other means. See SPARK-14602 for one reason of why this is needed.
+    distCacheMgr.updateConfiguration(sparkConf)
+
+    // Upload the conf archive to HDFS manually, and record its location in the configuration.
+    // This will allow the AM to know where the conf archive is in HDFS, so that it can be
+    // distributed to the containers.
+    //
+    // This code forces the archive to be copied, so that unit tests pass (since in that case both
+    // file systems are the same and the archive wouldn't normally be copied). In most (all?)
+    // deployments, the archive would be copied anyway, since it's a temp file in the local file
+    // system.
+    val remoteConfArchivePath = new Path(destDir, LOCALIZED_CONF_ARCHIVE)
+    val remoteFs = FileSystem.get(remoteConfArchivePath.toUri(), hadoopConf)
+    sparkConf.set(CACHED_CONF_ARCHIVE, remoteConfArchivePath.toString())
+
+    val localConfArchive = new Path(createConfArchive().toURI())
+    copyFileToRemote(destDir, localConfArchive, replication, force = true,
+      destName = Some(LOCALIZED_CONF_ARCHIVE))
+
+    // Manually add the config archive to the cache manager so that the AM is launched with
+    // the proper files set up.
+    distCacheMgr.addResource(
+      remoteFs, hadoopConf, remoteConfArchivePath, localResources, LocalResourceType.ARCHIVE,
+      LOCALIZED_CONF_DIR, statCache, appMasterOnly = false)
+
+    // Clear the cache-related entries from the configuration to avoid them polluting the
+    // UI's environment page. This works for client mode; for cluster mode, this is handled
+    // by the AM.
+    CACHE_CONFIGS.foreach(sparkConf.remove)
+
+    localResources
+  }
+
+  /**
+   * Create an archive with the config files for distribution.
+   *
+   * These will be used by AM and executors. The files are zipped and added to the job as an
+   * archive, so that YARN will explode it when distributing to AM and executors. This directory
+   * is then added to the classpath of AM and executor process, just to make sure that everybody
+   * is using the same default config.
+   *
+   * This follows the order of precedence set by the startup scripts, in which HADOOP_CONF_DIR
+   * shows up in the classpath before YARN_CONF_DIR.
+   *
+   * Currently this makes a shallow copy of the conf directory. If there are cases where a
+   * Hadoop config directory contains subdirectories, this code will have to be fixed.
+   *
+   * The archive also contains some Spark configuration. Namely, it saves the contents of
+   * SparkConf in a file to be loaded by the AM process.
+   */
+  private def createConfArchive(): File = {
+    val hadoopConfFiles = new HashMap[String, File]()
+
+    // Uploading $SPARK_CONF_DIR/log4j.properties file to the distributed cache to make sure that
+    // the executors will use the latest configurations instead of the default values. This is
+    // required when user changes log4j.properties directly to set the log configurations. If
+    // configuration file is provided through --files then executors will be taking configurations
+    // from --files instead of $SPARK_CONF_DIR/log4j.properties.
+
+    // Also uploading metrics.properties to distributed cache if exists in classpath.
+    // If user specify this file using --files then executors will use the one
+    // from --files instead.
+    for { prop <- Seq("log4j.properties", "metrics.properties")
+          url <- Option(Utils.getContextOrSparkClassLoader.getResource(prop))
+          if url.getProtocol == "file" } {
+      hadoopConfFiles(prop) = new File(url.getPath)
+    }
+
+    Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey =>
+      sys.env.get(envKey).foreach { path =>
+        val dir = new File(path)
+        if (dir.isDirectory()) {
+          val files = dir.listFiles()
+          if (files == null) {
+            logWarning("Failed to list files under directory " + dir)
+          } else {
+            files.foreach { file =>
+              if (file.isFile && !hadoopConfFiles.contains(file.getName())) {
+                hadoopConfFiles(file.getName()) = file
+              }
+            }
+          }
+        }
+      }
+    }
+
+    val confArchive = File.createTempFile(LOCALIZED_CONF_DIR, ".zip",
+      new File(Utils.getLocalDir(sparkConf)))
+    val confStream = new ZipOutputStream(new FileOutputStream(confArchive))
+
+    try {
+      confStream.setLevel(0)
+      hadoopConfFiles.foreach { case (name, file) =>
+        if (file.canRead()) {
+          confStream.putNextEntry(new ZipEntry(name))
+          Files.copy(file, confStream)
+          confStream.closeEntry()
+        }
+      }
+
+      // Save Spark configuration to a file in the archive.
+      val props = new Properties()
+      sparkConf.getAll.foreach { case (k, v) => props.setProperty(k, v) }
+      confStream.putNextEntry(new ZipEntry(SPARK_CONF_FILE))
+      val writer = new OutputStreamWriter(confStream, StandardCharsets.UTF_8)
+      props.store(writer, "Spark configuration.")
+      writer.flush()
+      confStream.closeEntry()
+    } finally {
+      confStream.close()
+    }
+    confArchive
+  }
+
+  /**
+   * Set up the environment for launching our ApplicationMaster container.
+   */
+  private def setupLaunchEnv(
+      stagingDirPath: Path,
+      pySparkArchives: Seq[String]): HashMap[String, String] = {
+    logInfo("Setting up the launch environment for our AM container")
+    val env = new HashMap[String, String]()
+    populateClasspath(args, yarnConf, sparkConf, env, sparkConf.get(DRIVER_CLASS_PATH))
+    env("SPARK_YARN_MODE") = "true"
+    env("SPARK_YARN_STAGING_DIR") = stagingDirPath.toString
+    env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName()
+    if (loginFromKeytab) {
+      val credentialsFile = "credentials-" + UUID.randomUUID().toString
+      sparkConf.set(CREDENTIALS_FILE_PATH, new Path(stagingDirPath, credentialsFile).toString)
+      logInfo(s"Credentials file set to: $credentialsFile")
+    }
+
+    // Pick up any environment variables for the AM provided through spark.yarn.appMasterEnv.*
+    val amEnvPrefix = "spark.yarn.appMasterEnv."
+    sparkConf.getAll
+      .filter { case (k, v) => k.startsWith(amEnvPrefix) }
+      .map { case (k, v) => (k.substring(amEnvPrefix.length), v) }
+      .foreach { case (k, v) => YarnSparkHadoopUtil.addPathToEnvironment(env, k, v) }
+
+    // Keep this for backwards compatibility but users should move to the config
+    sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs =>
+    // Allow users to specify some environment variables.
+      YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs)
+      // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments.
+      env("SPARK_YARN_USER_ENV") = userEnvs
+    }
+
+    // If pyFiles contains any .py files, we need to add LOCALIZED_PYTHON_DIR to the PYTHONPATH
+    // of the container processes too. Add all non-.py files directly to PYTHONPATH.
+    //
+    // NOTE: the code currently does not handle .py files defined with a "local:" scheme.
+    val pythonPath = new ListBuffer[String]()
+    val (pyFiles, pyArchives) = sparkConf.get(PY_FILES).partition(_.endsWith(".py"))
+    if (pyFiles.nonEmpty) {
+      pythonPath += buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
+        LOCALIZED_PYTHON_DIR)
+    }
+    (pySparkArchives ++ pyArchives).foreach { path =>
+      val uri = Utils.resolveURI(path)
+      if (uri.getScheme != LOCAL_SCHEME) {
+        pythonPath += buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
+          new Path(uri).getName())
+      } else {
+        pythonPath += uri.getPath()
+      }
+    }
+
+    // Finally, update the Spark config to propagate PYTHONPATH to the AM and executors.
+    if (pythonPath.nonEmpty) {
+      val pythonPathStr = (sys.env.get("PYTHONPATH") ++ pythonPath)
+        .mkString(YarnSparkHadoopUtil.getClassPathSeparator)
+      env("PYTHONPATH") = pythonPathStr
+      sparkConf.setExecutorEnv("PYTHONPATH", pythonPathStr)
+    }
+
+    // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to
+    // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's
+    // SparkContext will not let that set spark* system properties, which is expected behavior for
+    // Yarn clients. So propagate it through the environment.
+    //
+    // Note that to warn the user about the deprecation in cluster mode, some code from
+    // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition
+    // described above).
+    if (isClusterMode) {
+      sys.env.get("SPARK_JAVA_OPTS").foreach { value =>
+        val warning =
+          s"""
+            |SPARK_JAVA_OPTS was detected (set to '$value').
+            |This is deprecated in Spark 1.0+.
+            |
+            |Please instead use:
+            | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application
+            | - ./spark-submit with --driver-java-options to set -X options for a driver
+            | - spark.executor.extraJavaOptions to set -X options for executors
+          """.stripMargin
+        logWarning(warning)
+        for (proc <- Seq("driver", "executor")) {
+          val key = s"spark.$proc.extraJavaOptions"
+          if (sparkConf.contains(key)) {
+            throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.")
+          }
+        }
+        env("SPARK_JAVA_OPTS") = value
+      }
+      // propagate PYSPARK_DRIVER_PYTHON and PYSPARK_PYTHON to driver in cluster mode
+      Seq("PYSPARK_DRIVER_PYTHON", "PYSPARK_PYTHON").foreach { envname =>
+        if (!env.contains(envname)) {
+          sys.env.get(envname).foreach(env(envname) = _)
+        }
+      }
+    }
+
+    sys.env.get(ENV_DIST_CLASSPATH).foreach { dcp =>
+      env(ENV_DIST_CLASSPATH) = dcp
+    }
+
+    env
+  }
+
+  /**
+   * Set up a ContainerLaunchContext to launch our ApplicationMaster container.
+   * This sets up the launch environment, java options, and the command for launching the AM.
+   */
+  private def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse)
+    : ContainerLaunchContext = {
+    logInfo("Setting up container launch context for our AM")
+    val appId = newAppResponse.getApplicationId
+    val appStagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId))
+    val pySparkArchives =
+      if (sparkConf.get(IS_PYTHON_APP)) {
+        findPySparkArchives()
+      } else {
+        Nil
+      }
+    val launchEnv = setupLaunchEnv(appStagingDirPath, pySparkArchives)
+    val localResources = prepareLocalResources(appStagingDirPath, pySparkArchives)
+
+    val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
+    amContainer.setLocalResources(localResources.asJava)
+    amContainer.setEnvironment(launchEnv.asJava)
+
+    val javaOpts = ListBuffer[String]()
+
+    // Set the environment variable through a command prefix
+    // to append to the existing value of the variable
+    var prefixEnv: Option[String] = None
+
+    // Add Xmx for AM memory
+    javaOpts += "-Xmx" + amMemory + "m"
+
+    val tmpDir = new Path(
+      YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
+      YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR
+    )
+    javaOpts += "-Djava.io.tmpdir=" + tmpDir
+
+    // TODO: Remove once cpuset version is pushed out.
+    // The context is, default gc for server class machines ends up using all cores to do gc -
+    // hence if there are multiple containers in same node, Spark GC affects all other containers'
+    // performance (which can be that of other Spark containers)
+    // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in
+    // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset
+    // of cores on a node.
+    val useConcurrentAndIncrementalGC = launchEnv.get("SPARK_USE_CONC_INCR_GC").exists(_.toBoolean)
+    if (useConcurrentAndIncrementalGC) {
+      // In our expts, using (default) throughput collector has severe perf ramifications in
+      // multi-tenant machines
+      javaOpts += "-XX:+UseConcMarkSweepGC"
+      javaOpts += "-XX:MaxTenuringThreshold=31"
+      javaOpts += "-XX:SurvivorRatio=8"
+      javaOpts += "-XX:+CMSIncrementalMode"
+      javaOpts += "-XX:+CMSIncrementalPacing"
+      javaOpts += "-XX:CMSIncrementalDutyCycleMin=0"
+      javaOpts += "-XX:CMSIncrementalDutyCycle=10"
+    }
+
+    // Include driver-specific java options if we are launching a driver
+    if (isClusterMode) {
+      val driverOpts = sparkConf.get(DRIVER_JAVA_OPTIONS).orElse(sys.env.get("SPARK_JAVA_OPTS"))
+      driverOpts.foreach { opts =>
+        javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
+      }
+      val libraryPaths = Seq(sparkConf.get(DRIVER_LIBRARY_PATH),
+        sys.props.get("spark.driver.libraryPath")).flatten
+      if (libraryPaths.nonEmpty) {
+        prefixEnv = Some(getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(libraryPaths)))
+      }
+      if (sparkConf.get(AM_JAVA_OPTIONS).isDefined) {
+        logWarning(s"${AM_JAVA_OPTIONS.key} will not take effect in cluster mode")
+      }
+    } else {
+      // Validate and include yarn am specific java options in yarn-client mode.
+      sparkConf.get(AM_JAVA_OPTIONS).foreach { opts =>
+        if (opts.contains("-Dspark")) {
+          val msg = s"${AM_JAVA_OPTIONS.key} is not allowed to set Spark options (was '$opts')."
+          throw new SparkException(msg)
+        }
+        if (opts.contains("-Xmx")) {
+          val msg = s"${AM_JAVA_OPTIONS.key} is not allowed to specify max heap memory settings " +
+            s"(was '$opts'). Use spark.yarn.am.memory instead."
+          throw new SparkException(msg)
+        }
+        javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
+      }
+      sparkConf.get(AM_LIBRARY_PATH).foreach { paths =>
+        prefixEnv = Some(getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(Seq(paths))))
+      }
+    }
+
+    // For log4j configuration to reference
+    javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR)
+    YarnCommandBuilderUtils.addPermGenSizeOpt(javaOpts)
+
+    val userClass =
+      if (isClusterMode) {
+        Seq("--class", YarnSparkHadoopUtil.escapeForShell(args.userClass))
+      } else {
+        Nil
+      }
+    val userJar =
+      if (args.userJar != null) {
+        Seq("--jar", args.userJar)
+      } else {
+        Nil
+      }
+    val primaryPyFile =
+      if (isClusterMode && args.primaryPyFile != null) {
+        Seq("--primary-py-file", new Path(args.primaryPyFile).getName())
+      } else {
+        Nil
+      }
+    val primaryRFile =
+      if (args.primaryRFile != null) {
+        Seq("--primary-r-file", args.primaryRFile)
+      } else {
+        Nil
+      }
+    val amClass =
+      if (isClusterMode) {
+        Utils.classForName("org.apache.spark.deploy.yarn.ApplicationMaster").getName
+      } else {
+        Utils.classForName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName
+      }
+    if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) {
+      args.userArgs = ArrayBuffer(args.primaryRFile) ++ args.userArgs
+    }
+    val userArgs = args.userArgs.flatMap { arg =>
+      Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg))
+    }
+    val amArgs =
+      Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ primaryRFile ++
+        userArgs ++ Seq(
+          "--properties-file", buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
+            LOCALIZED_CONF_DIR, SPARK_CONF_FILE))
+
+    // Command for the ApplicationMaster
+    val commands = prefixEnv ++ Seq(
+        YarnSparkHadoopUtil.expandEnvironment(Environment.JAVA_HOME) + "/bin/java", "-server"
+      ) ++
+      javaOpts ++ amArgs ++
+      Seq(
+        "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout",
+        "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+
+    // TODO: it would be nicer to just make sure there are no null commands here
+    val printableCommands = commands.map(s => if (s == null) "null" else s).toList
+    amContainer.setCommands(printableCommands.asJava)
+
+    logDebug("===============================================================================")
+    logDebug("YARN AM launch context:")
+    logDebug(s"    user class: ${Option(args.userClass).getOrElse("N/A")}")
+    logDebug("    env:")
+    launchEnv.foreach { case (k, v) => logDebug(s"        $k -> $v") }
+    logDebug("    resources:")
+    localResources.foreach { case (k, v) => logDebug(s"        $k -> $v")}
+    logDebug("    command:")
+    logDebug(s"        ${printableCommands.mkString(" ")}")
+    logDebug("===============================================================================")
+
+    // send the acl settings into YARN to control who has access via YARN interfaces
+    val securityManager = new SecurityManager(sparkConf)
+    amContainer.setApplicationACLs(
+      YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager).asJava)
+    setupSecurityToken(amContainer)
+    amContainer
+  }
+
+  def setupCredentials(): Unit = {
+    loginFromKeytab = sparkConf.contains(PRINCIPAL.key)
+    if (loginFromKeytab) {
+      principal = sparkConf.get(PRINCIPAL).get
+      keytab = sparkConf.get(KEYTAB).orNull
+
+      require(keytab != null, "Keytab must be specified when principal is specified.")
+      logInfo("Attempting to login to the Kerberos" +
+        s" using principal: $principal and keytab: $keytab")
+      val f = new File(keytab)
+      // Generate a file name that can be used for the keytab file, that does not conflict
+      // with any user file.
+      val keytabFileName = f.getName + "-" + UUID.randomUUID().toString
+      sparkConf.set(KEYTAB.key, keytabFileName)
+      sparkConf.set(PRINCIPAL.key, principal)
+    }
+    // Defensive copy of the credentials
+    credentials = new Credentials(UserGroupInformation.getCurrentUser.getCredentials)
+  }
+
+  /**
+   * Report the state of an application until it has exited, either successfully or
+   * due to some failure, then return a pair of the yarn application state (FINISHED, FAILED,
+   * KILLED, or RUNNING) and the final application state (UNDEFINED, SUCCEEDED, FAILED,
+   * or KILLED).
+   *
+   * @param appId ID of the application to monitor.
+   * @param returnOnRunning Whether to also return the application state when it is RUNNING.
+   * @param logApplicationReport Whether to log details of the application report every iteration.
+   * @return A pair of the yarn application state and the final application state.
+   */
+  def monitorApplication(
+      appId: ApplicationId,
+      returnOnRunning: Boolean = false,
+      logApplicationReport: Boolean = true): (YarnApplicationState, FinalApplicationStatus) = {
+    val interval = sparkConf.get(REPORT_INTERVAL)
+    var lastState: YarnApplicationState = null
+    while (true) {
+      Thread.sleep(interval)
+      val report: ApplicationReport =
+        try {
+          getApplicationReport(appId)
+        } catch {
+          case e: ApplicationNotFoundException =>
+            logError(s"Application $appId not found.")
+            cleanupStagingDir(appId)
+            return (YarnApplicationState.KILLED, FinalApplicationStatus.KILLED)
+          case NonFatal(e) =>
+            logError(s"Failed to contact YARN for application $appId.", e)
+            // Don't necessarily clean up staging dir because status is unknown
+            return (YarnApplicationState.FAILED, FinalApplicationStatus.FAILED)
+        }
+      val state = report.getYarnApplicationState
+
+      if (logApplicationReport) {
+        logInfo(s"Application report for $appId (state: $state)")
+
+        // If DEBUG is enabled, log report details every iteration
+        // Otherwise, log them every time the application changes state
+        if (log.isDebugEnabled) {
+          logDebug(formatReportDetails(report))
+        } else if (lastState != state) {
+          logInfo(formatReportDetails(report))
+        }
+      }
+
+      if (lastState != state) {
+        state match {
+          case YarnApplicationState.RUNNING =>
+            reportLauncherState(SparkAppHandle.State.RUNNING)
+          case YarnApplicationState.FINISHED =>
+            report.getFinalApplicationStatus match {
+              case FinalApplicationStatus.FAILED =>
+                reportLauncherState(SparkAppHandle.State.FAILED)
+              case FinalApplicationStatus.KILLED =>
+                reportLauncherState(SparkAppHandle.State.KILLED)
+              case _ =>
+                reportLauncherState(SparkAppHandle.State.FINISHED)
+            }
+          case YarnApplicationState.FAILED =>
+            reportLauncherState(SparkAppHandle.State.FAILED)
+          case YarnApplicationState.KILLED =>
+            reportLauncherState(SparkAppHandle.State.KILLED)
+          case _ =>
+        }
+      }
+
+      if (state == YarnApplicationState.FINISHED ||
+        state == YarnApplicationState.FAILED ||
+        state == YarnApplicationState.KILLED) {
+        cleanupStagingDir(appId)
+        return (state, report.getFinalApplicationStatus)
+      }
+
+      if (returnOnRunning && state == YarnApplicationState.RUNNING) {
+        return (state, report.getFinalApplicationStatus)
+      }
+
+      lastState = state
+    }
+
+    // Never reached, but keeps compiler happy
+    throw new SparkException("While loop is depleted! This should never happen...")
+  }
+
+  private def formatReportDetails(report: ApplicationReport): String = {
+    val details = Seq[(String, String)](
+      ("client token", getClientToken(report)),
+      ("diagnostics", report.getDiagnostics),
+      ("ApplicationMaster host", report.getHost),
+      ("ApplicationMaster RPC port", report.getRpcPort.toString),
+      ("queue", report.getQueue),
+      ("start time", report.getStartTime.toString),
+      ("final status", report.getFinalApplicationStatus.toString),
+      ("tracking URL", report.getTrackingUrl),
+      ("user", report.getUser)
+    )
+
+    // Use more loggable format if value is null or empty
+    details.map { case (k, v) =>
+      val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A")
+      s"\n\t $k: $newValue"
+    }.mkString("")
+  }
+
+  /**
+   * Submit an application to the ResourceManager.
+   * If set spark.yarn.submit.waitAppCompletion to true, it will stay alive
+   * reporting the application's status until the application has exited for any reason.
+   * Otherwise, the client process will exit after submission.
+   * If the application finishes with a failed, killed, or undefined status,
+   * throw an appropriate SparkException.
+   */
+  def run(): Unit = {
+    this.appId = submitApplication()
+    if (!launcherBackend.isConnected() && fireAndForget) {
+      val report = getApplicationReport(appId)
+      val state = report.getYarnApplicationState
+      logInfo(s"Application report for $appId (state: $state)")
+      logInfo(formatReportDetails(report))
+      if (state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) {
+        throw new SparkException(s"Application $appId finished with status: $state")
+      }
+    } else {
+      val (yarnApplicationState, finalApplicationStatus) = monitorApplication(appId)
+      if (yarnApplicationState == YarnApplicationState.FAILED ||
+        finalApplicationStatus == FinalApplicationStatus.FAILED) {
+        throw new SparkException(s"Application $appId finished with failed status")
+      }
+      if (yarnApplicationState == YarnApplicationState.KILLED ||
+        finalApplicationStatus == FinalApplicationStatus.KILLED) {
+        throw new SparkException(s"Application $appId is killed")
+      }
+      if (finalApplicationStatus == FinalApplicationStatus.UNDEFINED) {
+        throw new SparkException(s"The final status of application $appId is undefined")
+      }
+    }
+  }
+
+  private def findPySparkArchives(): Seq[String] = {
+    sys.env.get("PYSPARK_ARCHIVES_PATH")
+      .map(_.split(",").toSeq)
+      .getOrElse {
+        val pyLibPath = Seq(sys.env("SPARK_HOME"), "python", "lib").mkString(File.separator)
+        val pyArchivesFile = new File(pyLibPath, "pyspark.zip")
+        require(pyArchivesFile.exists(),
+          s"$pyArchivesFile not found; cannot run pyspark application in YARN mode.")
+        val py4jFile = new File(pyLibPath, "py4j-0.10.4-src.zip")
+        require(py4jFile.exists(),
+          s"$py4jFile not found; cannot run pyspark application in YARN mode.")
+        Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath())
+      }
+  }
+
+}
+
+private object Client extends Logging {
+
+  def main(argStrings: Array[String]) {
+    if (!sys.props.contains("SPARK_SUBMIT")) {
+      logWarning("WARNING: This client is deprecated and will be removed in a " +
+        "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"")
+    }
+
+    // Set an env variable indicating we are running in YARN mode.
+    // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes
+    System.setProperty("SPARK_YARN_MODE", "true")
+    val sparkConf = new SparkConf
+    // SparkSubmit would use yarn cache to distribute files & jars in yarn mode,
+    // so remove them from sparkConf here for yarn mode.
+    sparkConf.remove("spark.jars")
+    sparkConf.remove("spark.files")
+    val args = new ClientArguments(argStrings)
+    new Client(args, sparkConf).run()
+  }
+
+  // Alias for the user jar
+  val APP_JAR_NAME: String = "__app__.jar"
+
+  // URI scheme that identifies local resources
+  val LOCAL_SCHEME = "local"
+
+  // Staging directory for any temporary jars or files
+  val SPARK_STAGING: String = ".sparkStaging"
+
+
+  // Staging directory is private! -> rwx--------
+  val STAGING_DIR_PERMISSION: FsPermission =
+    FsPermission.createImmutable(Integer.parseInt("700", 8).toShort)
+
+  // App files are world-wide readable and owner writable -> rw-r--r--
+  val APP_FILE_PERMISSION: FsPermission =
+    FsPermission.createImmutable(Integer.parseInt("644", 8).toShort)
+
+  // Distribution-defined classpath to add to processes
+  val ENV_DIST_CLASSPATH = "SPARK_DIST_CLASSPATH"
+
+  // Subdirectory where the user's Spark and Hadoop config files will be placed.
+  val LOCALIZED_CONF_DIR = "__spark_conf__"
+
+  // File containing the conf archive in the AM. See prepareLocalResources().
+  val LOCALIZED_CONF_ARCHIVE = LOCALIZED_CONF_DIR + ".zip"
+
+  // Name of the file in the conf archive containing Spark configuration.
+  val SPARK_CONF_FILE = "__spark_conf__.properties"
+
+  // Subdirectory where the user's python files (not archives) will be placed.
+  val LOCALIZED_PYTHON_DIR = "__pyfiles__"
+
+  // Subdirectory where Spark libraries will be placed.
+  val LOCALIZED_LIB_DIR = "__spark_libs__"
+
+  /**
+   * Return the path to the given application's staging directory.
+   */
+  private def getAppStagingDir(appId: ApplicationId): String = {
+    buildPath(SPARK_STAGING, appId.toString())
+  }
+
+  /**
+   * Populate the classpath entry in the given environment map with any application
+   * classpath specified through the Hadoop and Yarn configurations.
+   */
+  private[yarn] def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String])
+    : Unit = {
+    val classPathElementsToAdd = getYarnAppClasspath(conf) ++ getMRAppClasspath(conf)
+    for (c <- classPathElementsToAdd.flatten) {
+      YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, c.trim)
+    }
+  }
+
+  private def getYarnAppClasspath(conf: Configuration): Option[Seq[String]] =
+    Option(conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) match {
+      case Some(s) => Some(s.toSeq)
+      case None => getDefaultYarnApplicationClasspath
+    }
+
+  private def getMRAppClasspath(conf: Configuration): Option[Seq[String]] =
+    Option(conf.getStrings("mapreduce.application.classpath")) match {
+      case Some(s) => Some(s.toSeq)
+      case None => getDefaultMRApplicationClasspath
+    }
+
+  private[yarn] def getDefaultYarnApplicationClasspath: Option[Seq[String]] = {
+    val triedDefault = Try[Seq[String]] {
+      val field = classOf[YarnConfiguration].getField("DEFAULT_YARN_APPLICATION_CLASSPATH")
+      val value = field.get(null).asInstanceOf[Array[String]]
+      value.toSeq
+    } recoverWith {
+      case e: NoSuchFieldException => Success(Seq.empty[String])
+    }
+
+    triedDefault match {
+      case f: Failure[_] =>
+        logError("Unable to obtain the default YARN Application classpath.", f.exception)
+      case s: Success[Seq[String]] =>
+        logDebug(s"Using the default YARN application classpath: ${s.get.mkString(",")}")
+    }
+
+    triedDefault.toOption
+  }
+
+  private[yarn] def getDefaultMRApplicationClasspath: Option[Seq[String]] = {
+    val triedDefault = Try[Seq[String]] {
+      val field = classOf[MRJobConfig].getField("DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH")
+      StringUtils.getStrings(field.get(null).asInstanceOf[String]).toSeq
+    } recoverWith {
+      case e: NoSuchFieldException => Success(Seq.empty[String])
+    }
+
+    triedDefault match {
+      case f: Failure[_] =>
+        logError("Unable to obtain the default MR Application classpath.", f.exception)
+      case s: Success[Seq[String]] =>
+        logDebug(s"Using the default MR application classpath: ${s.get.mkString(",")}")
+    }
+
+    triedDefault.toOption
+  }
+
+  /**
+   * Populate the classpath entry in the given environment map.
+   *
+   * User jars are generally not added to the JVM's system classpath; those are handled by the AM
+   * and executor backend. When the deprecated `spark.yarn.user.classpath.first` is used, user jars
+   * are included in the system classpath, though. The extra class path and other uploaded files are
+   * always made available through the system class path.
+   *
+   * @param args Client arguments (when starting the AM) or null (when starting executors).
+   */
+  private[yarn] def populateClasspath(
+      args: ClientArguments,
+      conf: Configuration,
+      sparkConf: SparkConf,
+      env: HashMap[String, String],
+      extraClassPath: Option[String] = None): Unit = {
+    extraClassPath.foreach { cp =>
+      addClasspathEntry(getClusterPath(sparkConf, cp), env)
+    }
+
+    addClasspathEntry(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), env)
+
+    addClasspathEntry(
+      YarnSparkHadoopUtil.expandEnvironment(Environment.PWD) + Path.SEPARATOR +
+        LOCALIZED_CONF_DIR, env)
+
+    if (sparkConf.get(USER_CLASS_PATH_FIRST)) {
+      // in order to properly add the app jar when user classpath is first
+      // we have to do the mainJar separate in order to send the right thing
+      // into addFileToClasspath
+      val mainJar =
+        if (args != null) {
+          getMainJarUri(Option(args.userJar))
+        } else {
+          getMainJarUri(sparkConf.get(APP_JAR))
+        }
+      mainJar.foreach(addFileToClasspath(sparkConf, conf, _, APP_JAR_NAME, env))
+
+      val secondaryJars =
+        if (args != null) {
+          getSecondaryJarUris(Option(sparkConf.get(JARS_TO_DISTRIBUTE)))
+        } else {
+          getSecondaryJarUris(sparkConf.get(SECONDARY_JARS))
+        }
+      secondaryJars.foreach { x =>
+        addFileToClasspath(sparkConf, conf, x, null, env)
+      }
+    }
+
+    // Add the Spark jars to the classpath, depending on how they were distributed.
+    addClasspathEntry(buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
+      LOCALIZED_LIB_DIR, "*"), env)
+    if (!sparkConf.get(SPARK_ARCHIVE).isDefined) {
+      sparkConf.get(SPARK_JARS).foreach { jars =>
+        jars.filter(isLocalUri).foreach { jar =>
+          addClasspathEntry(getClusterPath(sparkConf, jar), env)
+        }
+      }
+    }
+
+    populateHadoopClasspath(conf, env)
+    sys.env.get(ENV_DIST_CLASSPATH).foreach { cp =>
+      addClasspathEntry(getClusterPath(sparkConf, cp), env)
+    }
+  }
+
+  /**
+   * Returns a list of URIs representing the user classpath.
+   *
+   * @param conf Spark configuration.
+   */
+  def getUserClasspath(conf: SparkConf): Array[URI] = {
+    val mainUri = getMainJarUri(conf.get(APP_JAR))
+    val secondaryUris = getSecondaryJarUris(conf.get(SECONDARY_JARS))
+    (mainUri ++ secondaryUris).toArray
+  }
+
+  private def getMainJarUri(mainJar: Option[String]): Option[URI] = {
+    mainJar.flatMap { path =>
+      val uri = Utils.resolveURI(path)
+      if (uri.getScheme == LOCAL_SCHEME) Some(uri) else None
+    }.orElse(Some(new URI(APP_JAR_NAME)))
+  }
+
+  private def getSecondaryJarUris(secondaryJars: Option[Seq[String]]): Seq[URI] = {
+    secondaryJars.getOrElse(Nil).map(new URI(_))
+  }
+
+  /**
+   * Adds the given path to the classpath, handling "local:" URIs correctly.
+   *
+   * If an alternate name for the file is given, and it's not a "local:" file, the alternate
+   * name will be added to the classpath (relative to the job's work directory).
+   *
+   * If not a "local:" file and no alternate name, the linkName will be added to the classpath.
+   *
+   * @param conf        Spark configuration.
+   * @param hadoopConf  Hadoop configuration.
+   * @param uri         URI to add to classpath (optional).
+   * @param fileName    Alternate name for the file (optional).
+   * @param env         Map holding the environment variables.
+   */
+  private def addFileToClasspath(
+      conf: SparkConf,
+      hadoopConf: Configuration,
+      uri: URI,
+      fileName: String,
+      env: HashMap[String, String]): Unit = {
+    if (uri != null && uri.getScheme == LOCAL_SCHEME) {
+      addClasspathEntry(getClusterPath(conf, uri.getPath), env)
+    } else if (fileName != null) {
+      addClasspathEntry(buildPath(
+        YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), fileName), env)
+    } else if (uri != null) {
+      val localPath = getQualifiedLocalPath(uri, hadoopConf)
+      val linkName = Option(uri.getFragment()).getOrElse(localPath.getName())
+      addClasspathEntry(buildPath(
+        YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), linkName), env)
+    }
+  }
+
+  /**
+   * Add the given path to the classpath entry of the given environment map.
+   * If the classpath is already set, this appends the new path to the existing classpath.
+   */
+  private def addClasspathEntry(path: String, env: HashMap[String, String]): Unit =
+    YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, path)
+
+  /**
+   * Returns the path to be sent to the NM for a path that is valid on the gateway.
+   *
+   * This method uses two configuration values:
+   *
+   *  - spark.yarn.config.gatewayPath: a string that identifies a portion of the input path that may
+   *    only be valid in the gateway node.
+   *  - spark.yarn.config.replacementPath: a string with which to replace the gateway path. This may
+   *    contain, for example, env variable references, which will be expanded by the NMs when
+   *    starting containers.
+   *
+   * If either config is not available, the input path is returned.
+   */
+  def getClusterPath(conf: SparkConf, path: String): String = {
+    val localPath = conf.get(GATEWAY_ROOT_PATH)
+    val clusterPath = conf.get(REPLACEMENT_ROOT_PATH)
+    if (localPath != null && clusterPath != null) {
+      path.replace(localPath, clusterPath)
+    } else {
+      path
+    }
+  }
+
+  /**
+   * Return whether the two file systems are the same.
+   */
+  private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = {
+    val srcUri = srcFs.getUri()
+    val dstUri = destFs.getUri()
+    if (srcUri.getScheme() == null || srcUri.getScheme() != dstUri.getScheme()) {
+      return false
+    }
+
+    var srcHost = srcUri.getHost()
+    var dstHost = dstUri.getHost()
+
+    // In HA or when using viewfs, the host part of the URI may not actually be a host, but the
+    // name of the HDFS namespace. Those names won't resolve, so avoid even trying if they
+    // match.
+    if (srcHost != null && dstHost != null && srcHost != dstHost) {
+      try {
+        srcHost = InetAddress.getByName(srcHost).getCanonicalHostName()
+        dstHost = InetAddress.getByName(dstHost).getCanonicalHostName()
+      } catch {
+        case e: UnknownHostException =>
+          return false
+      }
+    }
+
+    Objects.equal(srcHost, dstHost) && srcUri.getPort() == dstUri.getPort()
+  }
+
+  /**
+   * Given a local URI, resolve it and return a qualified local path that corresponds to the URI.
+   * This is used for preparing local resources to be included in the container launch context.
+   */
+  private def getQualifiedLocalPath(localURI: URI, hadoopConf: Configuration): Path = {
+    val qualifiedURI =
+      if (localURI.getScheme == null) {
+        // If not specified, assume this is in the local filesystem to keep the behavior
+        // consistent with that of Hadoop
+        new URI(FileSystem.getLocal(hadoopConf).makeQualified(new Path(localURI)).toString)
+      } else {
+        localURI
+      }
+    new Path(qualifiedURI)
+  }
+
+  /**
+   * Whether to consider jars provided by the user to have precedence over the Spark jars when
+   * loading user classes.
+   */
+  def isUserClassPathFirst(conf: SparkConf, isDriver: Boolean): Boolean = {
+    if (isDriver) {
+      conf.get(DRIVER_USER_CLASS_PATH_FIRST)
+    } else {
+      conf.get(EXECUTOR_USER_CLASS_PATH_FIRST)
+    }
+  }
+
+  /**
+   * Joins all the path components using Path.SEPARATOR.
+   */
+  def buildPath(components: String*): String = {
+    components.mkString(Path.SEPARATOR)
+  }
+
+  /** Returns whether the URI is a "local:" URI. */
+  def isLocalUri(uri: String): Boolean = {
+    uri.startsWith(s"$LOCAL_SCHEME:")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
new file mode 100644
index 0000000..61c027e
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.yarn
+
+import scala.collection.mutable.ArrayBuffer
+
+// TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware !
+private[spark] class ClientArguments(args: Array[String]) {
+
+  var userJar: String = null
+  var userClass: String = null
+  var primaryPyFile: String = null
+  var primaryRFile: String = null
+  var userArgs: ArrayBuffer[String] = new ArrayBuffer[String]()
+
+  parseArgs(args.toList)
+
+  private def parseArgs(inputArgs: List[String]): Unit = {
+    var args = inputArgs
+
+    while (!args.isEmpty) {
+      args match {
+        case ("--jar") :: value :: tail =>
+          userJar = value
+          args = tail
+
+        case ("--class") :: value :: tail =>
+          userClass = value
+          args = tail
+
+        case ("--primary-py-file") :: value :: tail =>
+          primaryPyFile = value
+          args = tail
+
+        case ("--primary-r-file") :: value :: tail =>
+          primaryRFile = value
+          args = tail
+
+        case ("--arg") :: value :: tail =>
+          userArgs += value
+          args = tail
+
+        case Nil =>
+
+        case _ =>
+          throw new IllegalArgumentException(getUsageMessage(args))
+      }
+    }
+
+    if (primaryPyFile != null && primaryRFile != null) {
+      throw new IllegalArgumentException("Cannot have primary-py-file and primary-r-file" +
+        " at the same time")
+    }
+  }
+
+  private def getUsageMessage(unknownParam: List[String] = null): String = {
+    val message = if (unknownParam != null) s"Unknown/unsupported param $unknownParam\n" else ""
+    message +
+      s"""
+      |Usage: org.apache.spark.deploy.yarn.Client [options]
+      |Options:
+      |  --jar JAR_PATH           Path to your application's JAR file (required in yarn-cluster
+      |                           mode)
+      |  --class CLASS_NAME       Name of your application's main class (required)
+      |  --primary-py-file        A main Python file
+      |  --primary-r-file         A main R file
+      |  --arg ARG                Argument to be passed to your application's main class.
+      |                           Multiple invocations are possible, each will be passed in order.
+      """.stripMargin
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
new file mode 100644
index 0000000..dcc2288
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
@@ -0,0 +1,186 @@
+/*
+ * 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 java.net.URI
+
+import scala.collection.mutable.{HashMap, ListBuffer, Map}
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
+import org.apache.hadoop.fs.permission.FsAction
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+
+private case class CacheEntry(
+  uri: URI,
+  size: Long,
+  modTime: Long,
+  visibility: LocalResourceVisibility,
+  resType: LocalResourceType)
+
+/** Client side methods to setup the Hadoop distributed cache */
+private[spark] class ClientDistributedCacheManager() extends Logging {
+
+  private val distCacheEntries = new ListBuffer[CacheEntry]()
+
+  /**
+   * Add a resource to the list of distributed cache resources. This list can
+   * be sent to the ApplicationMaster and possibly the executors so that it can
+   * be downloaded into the Hadoop distributed cache for use by this application.
+   * Adds the LocalResource to the localResources HashMap passed in and saves
+   * the stats of the resources to they can be sent to the executors and verified.
+   *
+   * @param fs FileSystem
+   * @param conf Configuration
+   * @param destPath path to the resource
+   * @param localResources localResource hashMap to insert the resource into
+   * @param resourceType LocalResourceType
+   * @param link link presented in the distributed cache to the destination
+   * @param statCache cache to store the file/directory stats
+   * @param appMasterOnly Whether to only add the resource to the app master
+   */
+  def addResource(
+      fs: FileSystem,
+      conf: Configuration,
+      destPath: Path,
+      localResources: HashMap[String, LocalResource],
+      resourceType: LocalResourceType,
+      link: String,
+      statCache: Map[URI, FileStatus],
+      appMasterOnly: Boolean = false): Unit = {
+    val destStatus = fs.getFileStatus(destPath)
+    val amJarRsrc = Records.newRecord(classOf[LocalResource])
+    amJarRsrc.setType(resourceType)
+    val visibility = getVisibility(conf, destPath.toUri(), statCache)
+    amJarRsrc.setVisibility(visibility)
+    amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(destPath))
+    amJarRsrc.setTimestamp(destStatus.getModificationTime())
+    amJarRsrc.setSize(destStatus.getLen())
+    require(link != null && link.nonEmpty, "You must specify a valid link name.")
+    localResources(link) = amJarRsrc
+
+    if (!appMasterOnly) {
+      val uri = destPath.toUri()
+      val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link)
+      distCacheEntries += CacheEntry(pathURI, destStatus.getLen(), destStatus.getModificationTime(),
+        visibility, resourceType)
+    }
+  }
+
+  /**
+   * Writes down information about cached files needed in executors to the given configuration.
+   */
+  def updateConfiguration(conf: SparkConf): Unit = {
+    conf.set(CACHED_FILES, distCacheEntries.map(_.uri.toString))
+    conf.set(CACHED_FILES_SIZES, distCacheEntries.map(_.size))
+    conf.set(CACHED_FILES_TIMESTAMPS, distCacheEntries.map(_.modTime))
+    conf.set(CACHED_FILES_VISIBILITIES, distCacheEntries.map(_.visibility.name()))
+    conf.set(CACHED_FILES_TYPES, distCacheEntries.map(_.resType.name()))
+  }
+
+  /**
+   * Returns the local resource visibility depending on the cache file permissions
+   * @return LocalResourceVisibility
+   */
+  private[yarn] def getVisibility(
+      conf: Configuration,
+      uri: URI,
+      statCache: Map[URI, FileStatus]): LocalResourceVisibility = {
+    if (isPublic(conf, uri, statCache)) {
+      LocalResourceVisibility.PUBLIC
+    } else {
+      LocalResourceVisibility.PRIVATE
+    }
+  }
+
+  /**
+   * Returns a boolean to denote whether a cache file is visible to all (public)
+   * @return true if the path in the uri is visible to all, false otherwise
+   */
+  private def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = {
+    val fs = FileSystem.get(uri, conf)
+    val current = new Path(uri.getPath())
+    // the leaf level file should be readable by others
+    if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) {
+      return false
+    }
+    ancestorsHaveExecutePermissions(fs, current.getParent(), statCache)
+  }
+
+  /**
+   * Returns true if all ancestors of the specified path have the 'execute'
+   * permission set for all users (i.e. that other users can traverse
+   * the directory hierarchy to the given path)
+   * @return true if all ancestors have the 'execute' permission set for all users
+   */
+  private def ancestorsHaveExecutePermissions(
+      fs: FileSystem,
+      path: Path,
+      statCache: Map[URI, FileStatus]): Boolean = {
+    var current = path
+    while (current != null) {
+      // the subdirs in the path should have execute permissions for others
+      if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE, statCache)) {
+        return false
+      }
+      current = current.getParent()
+    }
+    true
+  }
+
+  /**
+   * Checks for a given path whether the Other permissions on it
+   * imply the permission in the passed FsAction
+   * @return true if the path in the uri is visible to all, false otherwise
+   */
+  private def checkPermissionOfOther(
+      fs: FileSystem,
+      path: Path,
+      action: FsAction,
+      statCache: Map[URI, FileStatus]): Boolean = {
+    val status = getFileStatus(fs, path.toUri(), statCache)
+    val perms = status.getPermission()
+    val otherAction = perms.getOtherAction()
+    otherAction.implies(action)
+  }
+
+  /**
+   * Checks to see if the given uri exists in the cache, if it does it
+   * returns the existing FileStatus, otherwise it stats the uri, stores
+   * it in the cache, and returns the FileStatus.
+   * @return FileStatus
+   */
+  private[yarn] def getFileStatus(
+      fs: FileSystem,
+      uri: URI,
+      statCache: Map[URI, FileStatus]): FileStatus = {
+    val stat = statCache.get(uri) match {
+      case Some(existstat) => existstat
+      case None =>
+        val newStat = fs.getFileStatus(new Path(uri))
+        statCache.put(uri, newStat)
+        newStat
+    }
+    stat
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
new file mode 100644
index 0000000..868c2ed
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
@@ -0,0 +1,266 @@
+/*
+ * 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 java.io.File
+import java.nio.ByteBuffer
+import java.util.Collections
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{HashMap, ListBuffer}
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.NMClient
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkException}
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.launcher.YarnCommandBuilderUtils
+import org.apache.spark.network.util.JavaUtils
+import org.apache.spark.util.Utils
+
+private[yarn] class ExecutorRunnable(
+    container: Option[Container],
+    conf: YarnConfiguration,
+    sparkConf: SparkConf,
+    masterAddress: String,
+    executorId: String,
+    hostname: String,
+    executorMemory: Int,
+    executorCores: Int,
+    appId: String,
+    securityMgr: SecurityManager,
+    localResources: Map[String, LocalResource]) extends Logging {
+
+  var rpc: YarnRPC = YarnRPC.create(conf)
+  var nmClient: NMClient = _
+
+  def run(): Unit = {
+    logDebug("Starting Executor Container")
+    nmClient = NMClient.createNMClient()
+    nmClient.init(conf)
+    nmClient.start()
+    startContainer()
+  }
+
+  def launchContextDebugInfo(): String = {
+    val commands = prepareCommand()
+    val env = prepareEnvironment()
+
+    s"""
+    |===============================================================================
+    |YARN executor launch context:
+    |  env:
+    |${Utils.redact(sparkConf, env.toSeq).map { case (k, v) => s"    $k -> $v\n" }.mkString}
+    |  command:
+    |    ${commands.mkString(" \\ \n      ")}
+    |
+    |  resources:
+    |${localResources.map { case (k, v) => s"    $k -> $v\n" }.mkString}
+    |===============================================================================""".stripMargin
+  }
+
+  def startContainer(): java.util.Map[String, ByteBuffer] = {
+    val ctx = Records.newRecord(classOf[ContainerLaunchContext])
+      .asInstanceOf[ContainerLaunchContext]
+    val env = prepareEnvironment().asJava
+
+    ctx.setLocalResources(localResources.asJava)
+    ctx.setEnvironment(env)
+
+    val credentials = UserGroupInformation.getCurrentUser().getCredentials()
+    val dob = new DataOutputBuffer()
+    credentials.writeTokenStorageToStream(dob)
+    ctx.setTokens(ByteBuffer.wrap(dob.getData()))
+
+    val commands = prepareCommand()
+
+    ctx.setCommands(commands.asJava)
+    ctx.setApplicationACLs(
+      YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr).asJava)
+
+    // If external shuffle service is enabled, register with the Yarn shuffle service already
+    // started on the NodeManager and, if authentication is enabled, provide it with our secret
+    // key for fetching shuffle files later
+    if (sparkConf.get(SHUFFLE_SERVICE_ENABLED)) {
+      val secretString = securityMgr.getSecretKey()
+      val secretBytes =
+        if (secretString != null) {
+          // This conversion must match how the YarnShuffleService decodes our secret
+          JavaUtils.stringToBytes(secretString)
+        } else {
+          // Authentication is not enabled, so just provide dummy metadata
+          ByteBuffer.allocate(0)
+        }
+      ctx.setServiceData(Collections.singletonMap("spark_shuffle", secretBytes))
+    }
+
+    // Send the start request to the ContainerManager
+    try {
+      nmClient.startContainer(container.get, ctx)
+    } catch {
+      case ex: Exception =>
+        throw new SparkException(s"Exception while starting container ${container.get.getId}" +
+          s" on host $hostname", ex)
+    }
+  }
+
+  private def prepareCommand(): List[String] = {
+    // Extra options for the JVM
+    val javaOpts = ListBuffer[String]()
+
+    // Set the environment variable through a command prefix
+    // to append to the existing value of the variable
+    var prefixEnv: Option[String] = None
+
+    // Set the JVM memory
+    val executorMemoryString = executorMemory + "m"
+    javaOpts += "-Xmx" + executorMemoryString
+
+    // Set extra Java options for the executor, if defined
+    sparkConf.get(EXECUTOR_JAVA_OPTIONS).foreach { opts =>
+      javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
+    }
+    sys.env.get("SPARK_JAVA_OPTS").foreach { opts =>
+      javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
+    }
+    sparkConf.get(EXECUTOR_LIBRARY_PATH).foreach { p =>
+      prefixEnv = Some(Client.getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(Seq(p))))
+    }
+
+    javaOpts += "-Djava.io.tmpdir=" +
+      new Path(
+        YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
+        YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR
+      )
+
+    // Certain configs need to be passed here because they are needed before the Executor
+    // registers with the Scheduler and transfers the spark configs. Since the Executor backend
+    // uses RPC to connect to the scheduler, the RPC settings are needed as well as the
+    // authentication settings.
+    sparkConf.getAll
+      .filter { case (k, v) => SparkConf.isExecutorStartupConf(k) }
+      .foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") }
+
+    // Commenting it out for now - so that people can refer to the properties if required. Remove
+    // it once cpuset version is pushed out.
+    // The context is, default gc for server class machines end up using all cores to do gc - hence
+    // if there are multiple containers in same node, spark gc effects all other containers
+    // performance (which can also be other spark containers)
+    // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
+    // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
+    // of cores on a node.
+    /*
+        else {
+          // If no java_opts specified, default to using -XX:+CMSIncrementalMode
+          // It might be possible that other modes/config is being done in
+          // spark.executor.extraJavaOptions, so we don't want to mess with it.
+          // In our expts, using (default) throughput collector has severe perf ramifications in
+          // multi-tenant machines
+          // The options are based on
+          // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use
+          // %20the%20Concurrent%20Low%20Pause%20Collector|outline
+          javaOpts += "-XX:+UseConcMarkSweepGC"
+          javaOpts += "-XX:+CMSIncrementalMode"
+          javaOpts += "-XX:+CMSIncrementalPacing"
+          javaOpts += "-XX:CMSIncrementalDutyCycleMin=0"
+          javaOpts += "-XX:CMSIncrementalDutyCycle=10"
+        }
+    */
+
+    // For log4j configuration to reference
+    javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR)
+    YarnCommandBuilderUtils.addPermGenSizeOpt(javaOpts)
+
+    val userClassPath = Client.getUserClasspath(sparkConf).flatMap { uri =>
+      val absPath =
+        if (new File(uri.getPath()).isAbsolute()) {
+          Client.getClusterPath(sparkConf, uri.getPath())
+        } else {
+          Client.buildPath(Environment.PWD.$(), uri.getPath())
+        }
+      Seq("--user-class-path", "file:" + absPath)
+    }.toSeq
+
+    YarnSparkHadoopUtil.addOutOfMemoryErrorArgument(javaOpts)
+    val commands = prefixEnv ++ Seq(
+      YarnSparkHadoopUtil.expandEnvironment(Environment.JAVA_HOME) + "/bin/java",
+      "-server") ++
+      javaOpts ++
+      Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend",
+        "--driver-url", masterAddress,
+        "--executor-id", executorId,
+        "--hostname", hostname,
+        "--cores", executorCores.toString,
+        "--app-id", appId) ++
+      userClassPath ++
+      Seq(
+        s"1>${ApplicationConstants.LOG_DIR_EXPANSION_VAR}/stdout",
+        s"2>${ApplicationConstants.LOG_DIR_EXPANSION_VAR}/stderr")
+
+    // TODO: it would be nicer to just make sure there are no null commands here
+    commands.map(s => if (s == null) "null" else s).toList
+  }
+
+  private def prepareEnvironment(): HashMap[String, String] = {
+    val env = new HashMap[String, String]()
+    Client.populateClasspath(null, conf, sparkConf, env, sparkConf.get(EXECUTOR_CLASS_PATH))
+
+    sparkConf.getExecutorEnv.foreach { case (key, value) =>
+      // This assumes each executor environment variable set here is a path
+      // This is kept for backward compatibility and consistency with hadoop
+      YarnSparkHadoopUtil.addPathToEnvironment(env, key, value)
+    }
+
+    // Keep this for backwards compatibility but users should move to the config
+    sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs =>
+      YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs)
+    }
+
+    // lookup appropriate http scheme for container log urls
+    val yarnHttpPolicy = conf.get(
+      YarnConfiguration.YARN_HTTP_POLICY_KEY,
+      YarnConfiguration.YARN_HTTP_POLICY_DEFAULT
+    )
+    val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://"
+
+    // Add log urls
+    container.foreach { c =>
+      sys.env.get("SPARK_USER").foreach { user =>
+        val containerId = ConverterUtils.toString(c.getId)
+        val address = c.getNodeHttpAddress
+        val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user"
+
+        env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=-4096"
+        env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=-4096"
+      }
+    }
+
+    System.getenv().asScala.filterKeys(_.startsWith("SPARK"))
+      .foreach { case (k, v) => env(k) = v }
+    env
+  }
+}


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


[02/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
deleted file mode 100644
index 7deaf0a..0000000
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
+++ /dev/null
@@ -1,462 +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.deploy.yarn
-
-import java.io.{File, FileInputStream, FileOutputStream}
-import java.net.URI
-import java.util.Properties
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.{HashMap => MutableHashMap}
-import scala.reflect.ClassTag
-import scala.util.Try
-
-import org.apache.commons.lang3.SerializationUtils
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.mapreduce.MRJobConfig
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse
-import org.apache.hadoop.yarn.api.records._
-import org.apache.hadoop.yarn.client.api.YarnClientApplication
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.yarn.util.Records
-import org.mockito.Matchers.{eq => meq, _}
-import org.mockito.Mockito._
-import org.scalatest.{BeforeAndAfterAll, Matchers}
-
-import org.apache.spark.{SparkConf, SparkFunSuite, TestUtils}
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.util.{ResetSystemProperties, SparkConfWithEnv, Utils}
-
-class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll
-  with ResetSystemProperties {
-
-  import Client._
-
-  var oldSystemProperties: Properties = null
-
-  override def beforeAll(): Unit = {
-    super.beforeAll()
-    oldSystemProperties = SerializationUtils.clone(System.getProperties)
-    System.setProperty("SPARK_YARN_MODE", "true")
-  }
-
-  override def afterAll(): Unit = {
-    try {
-      System.setProperties(oldSystemProperties)
-      oldSystemProperties = null
-    } finally {
-      super.afterAll()
-    }
-  }
-
-  test("default Yarn application classpath") {
-    getDefaultYarnApplicationClasspath should be(Some(Fixtures.knownDefYarnAppCP))
-  }
-
-  test("default MR application classpath") {
-    getDefaultMRApplicationClasspath should be(Some(Fixtures.knownDefMRAppCP))
-  }
-
-  test("resultant classpath for an application that defines a classpath for YARN") {
-    withAppConf(Fixtures.mapYARNAppConf) { conf =>
-      val env = newEnv
-      populateHadoopClasspath(conf, env)
-      classpath(env) should be(
-        flatten(Fixtures.knownYARNAppCP, getDefaultMRApplicationClasspath))
-    }
-  }
-
-  test("resultant classpath for an application that defines a classpath for MR") {
-    withAppConf(Fixtures.mapMRAppConf) { conf =>
-      val env = newEnv
-      populateHadoopClasspath(conf, env)
-      classpath(env) should be(
-        flatten(getDefaultYarnApplicationClasspath, Fixtures.knownMRAppCP))
-    }
-  }
-
-  test("resultant classpath for an application that defines both classpaths, YARN and MR") {
-    withAppConf(Fixtures.mapAppConf) { conf =>
-      val env = newEnv
-      populateHadoopClasspath(conf, env)
-      classpath(env) should be(flatten(Fixtures.knownYARNAppCP, Fixtures.knownMRAppCP))
-    }
-  }
-
-  private val SPARK = "local:/sparkJar"
-  private val USER = "local:/userJar"
-  private val ADDED = "local:/addJar1,local:/addJar2,/addJar3"
-
-  private val PWD =
-    if (classOf[Environment].getMethods().exists(_.getName == "$$")) {
-      "{{PWD}}"
-    } else if (Utils.isWindows) {
-      "%PWD%"
-    } else {
-      Environment.PWD.$()
-    }
-
-  test("Local jar URIs") {
-    val conf = new Configuration()
-    val sparkConf = new SparkConf()
-      .set(SPARK_JARS, Seq(SPARK))
-      .set(USER_CLASS_PATH_FIRST, true)
-      .set("spark.yarn.dist.jars", ADDED)
-    val env = new MutableHashMap[String, String]()
-    val args = new ClientArguments(Array("--jar", USER))
-
-    populateClasspath(args, conf, sparkConf, env)
-
-    val cp = env("CLASSPATH").split(":|;|<CPS>")
-    s"$SPARK,$USER,$ADDED".split(",").foreach({ entry =>
-      val uri = new URI(entry)
-      if (LOCAL_SCHEME.equals(uri.getScheme())) {
-        cp should contain (uri.getPath())
-      } else {
-        cp should not contain (uri.getPath())
-      }
-    })
-    cp should contain(PWD)
-    cp should contain (s"$PWD${Path.SEPARATOR}${LOCALIZED_CONF_DIR}")
-    cp should not contain (APP_JAR)
-  }
-
-  test("Jar path propagation through SparkConf") {
-    val conf = new Configuration()
-    val sparkConf = new SparkConf()
-      .set(SPARK_JARS, Seq(SPARK))
-      .set("spark.yarn.dist.jars", ADDED)
-    val client = createClient(sparkConf, args = Array("--jar", USER))
-    doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]),
-      any(classOf[Path]), anyShort(), anyBoolean(), any())
-
-    val tempDir = Utils.createTempDir()
-    try {
-      // Because we mocked "copyFileToRemote" above to avoid having to create fake local files,
-      // we need to create a fake config archive in the temp dir to avoid having
-      // prepareLocalResources throw an exception.
-      new FileOutputStream(new File(tempDir, LOCALIZED_CONF_ARCHIVE)).close()
-
-      client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil)
-      sparkConf.get(APP_JAR) should be (Some(USER))
-
-      // The non-local path should be propagated by name only, since it will end up in the app's
-      // staging dir.
-      val expected = ADDED.split(",")
-        .map(p => {
-          val uri = new URI(p)
-          if (LOCAL_SCHEME == uri.getScheme()) {
-            p
-          } else {
-            Option(uri.getFragment()).getOrElse(new File(p).getName())
-          }
-        })
-        .mkString(",")
-
-      sparkConf.get(SECONDARY_JARS) should be (Some(expected.split(",").toSeq))
-    } finally {
-      Utils.deleteRecursively(tempDir)
-    }
-  }
-
-  test("Cluster path translation") {
-    val conf = new Configuration()
-    val sparkConf = new SparkConf()
-      .set(SPARK_JARS, Seq("local:/localPath/spark.jar"))
-      .set(GATEWAY_ROOT_PATH, "/localPath")
-      .set(REPLACEMENT_ROOT_PATH, "/remotePath")
-
-    getClusterPath(sparkConf, "/localPath") should be ("/remotePath")
-    getClusterPath(sparkConf, "/localPath/1:/localPath/2") should be (
-      "/remotePath/1:/remotePath/2")
-
-    val env = new MutableHashMap[String, String]()
-    populateClasspath(null, conf, sparkConf, env, extraClassPath = Some("/localPath/my1.jar"))
-    val cp = classpath(env)
-    cp should contain ("/remotePath/spark.jar")
-    cp should contain ("/remotePath/my1.jar")
-  }
-
-  test("configuration and args propagate through createApplicationSubmissionContext") {
-    val conf = new Configuration()
-    // When parsing tags, duplicates and leading/trailing whitespace should be removed.
-    // Spaces between non-comma strings should be preserved as single tags. Empty strings may or
-    // may not be removed depending on the version of Hadoop being used.
-    val sparkConf = new SparkConf()
-      .set(APPLICATION_TAGS.key, ",tag1, dup,tag2 , ,multi word , dup")
-      .set(MAX_APP_ATTEMPTS, 42)
-      .set("spark.app.name", "foo-test-app")
-      .set(QUEUE_NAME, "staging-queue")
-    val args = new ClientArguments(Array())
-
-    val appContext = Records.newRecord(classOf[ApplicationSubmissionContext])
-    val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse])
-    val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext])
-
-    val client = new Client(args, conf, sparkConf)
-    client.createApplicationSubmissionContext(
-      new YarnClientApplication(getNewApplicationResponse, appContext),
-      containerLaunchContext)
-
-    appContext.getApplicationName should be ("foo-test-app")
-    appContext.getQueue should be ("staging-queue")
-    appContext.getAMContainerSpec should be (containerLaunchContext)
-    appContext.getApplicationType should be ("SPARK")
-    appContext.getClass.getMethods.filter(_.getName.equals("getApplicationTags")).foreach{ method =>
-      val tags = method.invoke(appContext).asInstanceOf[java.util.Set[String]]
-      tags should contain allOf ("tag1", "dup", "tag2", "multi word")
-      tags.asScala.count(_.nonEmpty) should be (4)
-    }
-    appContext.getMaxAppAttempts should be (42)
-  }
-
-  test("spark.yarn.jars with multiple paths and globs") {
-    val libs = Utils.createTempDir()
-    val single = Utils.createTempDir()
-    val jar1 = TestUtils.createJarWithFiles(Map(), libs)
-    val jar2 = TestUtils.createJarWithFiles(Map(), libs)
-    val jar3 = TestUtils.createJarWithFiles(Map(), single)
-    val jar4 = TestUtils.createJarWithFiles(Map(), single)
-
-    val jarsConf = Seq(
-      s"${libs.getAbsolutePath()}/*",
-      jar3.getPath(),
-      s"local:${jar4.getPath()}",
-      s"local:${single.getAbsolutePath()}/*")
-
-    val sparkConf = new SparkConf().set(SPARK_JARS, jarsConf)
-    val client = createClient(sparkConf)
-
-    val tempDir = Utils.createTempDir()
-    client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil)
-
-    assert(sparkConf.get(SPARK_JARS) ===
-      Some(Seq(s"local:${jar4.getPath()}", s"local:${single.getAbsolutePath()}/*")))
-
-    verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar1.toURI())), anyShort(),
-      anyBoolean(), any())
-    verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar2.toURI())), anyShort(),
-      anyBoolean(), any())
-    verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar3.toURI())), anyShort(),
-      anyBoolean(), any())
-
-    val cp = classpath(client)
-    cp should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*"))
-    cp should not contain (jar3.getPath())
-    cp should contain (jar4.getPath())
-    cp should contain (buildPath(single.getAbsolutePath(), "*"))
-  }
-
-  test("distribute jars archive") {
-    val temp = Utils.createTempDir()
-    val archive = TestUtils.createJarWithFiles(Map(), temp)
-
-    val sparkConf = new SparkConf().set(SPARK_ARCHIVE, archive.getPath())
-    val client = createClient(sparkConf)
-    client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil)
-
-    verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(archive.toURI())), anyShort(),
-      anyBoolean(), any())
-    classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*"))
-
-    sparkConf.set(SPARK_ARCHIVE, LOCAL_SCHEME + ":" + archive.getPath())
-    intercept[IllegalArgumentException] {
-      client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil)
-    }
-  }
-
-  test("distribute archive multiple times") {
-    val libs = Utils.createTempDir()
-    // Create jars dir and RELEASE file to avoid IllegalStateException.
-    val jarsDir = new File(libs, "jars")
-    assert(jarsDir.mkdir())
-    new FileOutputStream(new File(libs, "RELEASE")).close()
-
-    val userLib1 = Utils.createTempDir()
-    val testJar = TestUtils.createJarWithFiles(Map(), userLib1)
-
-    // Case 1:  FILES_TO_DISTRIBUTE and ARCHIVES_TO_DISTRIBUTE can't have duplicate files
-    val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
-      .set(FILES_TO_DISTRIBUTE, Seq(testJar.getPath))
-      .set(ARCHIVES_TO_DISTRIBUTE, Seq(testJar.getPath))
-
-    val client = createClient(sparkConf)
-    val tempDir = Utils.createTempDir()
-    intercept[IllegalArgumentException] {
-      client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil)
-    }
-
-    // Case 2: FILES_TO_DISTRIBUTE can't have duplicate files.
-    val sparkConfFiles = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
-      .set(FILES_TO_DISTRIBUTE, Seq(testJar.getPath, testJar.getPath))
-
-    val clientFiles = createClient(sparkConfFiles)
-    val tempDirForFiles = Utils.createTempDir()
-    intercept[IllegalArgumentException] {
-      clientFiles.prepareLocalResources(new Path(tempDirForFiles.getAbsolutePath()), Nil)
-    }
-
-    // Case 3: ARCHIVES_TO_DISTRIBUTE can't have duplicate files.
-    val sparkConfArchives = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
-      .set(ARCHIVES_TO_DISTRIBUTE, Seq(testJar.getPath, testJar.getPath))
-
-    val clientArchives = createClient(sparkConfArchives)
-    val tempDirForArchives = Utils.createTempDir()
-    intercept[IllegalArgumentException] {
-      clientArchives.prepareLocalResources(new Path(tempDirForArchives.getAbsolutePath()), Nil)
-    }
-
-    // Case 4: FILES_TO_DISTRIBUTE can have unique file.
-    val sparkConfFilesUniq = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
-      .set(FILES_TO_DISTRIBUTE, Seq(testJar.getPath))
-
-    val clientFilesUniq = createClient(sparkConfFilesUniq)
-    val tempDirForFilesUniq = Utils.createTempDir()
-    clientFilesUniq.prepareLocalResources(new Path(tempDirForFilesUniq.getAbsolutePath()), Nil)
-
-    // Case 5: ARCHIVES_TO_DISTRIBUTE can have unique file.
-    val sparkConfArchivesUniq = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
-      .set(ARCHIVES_TO_DISTRIBUTE, Seq(testJar.getPath))
-
-    val clientArchivesUniq = createClient(sparkConfArchivesUniq)
-    val tempDirArchivesUniq = Utils.createTempDir()
-    clientArchivesUniq.prepareLocalResources(new Path(tempDirArchivesUniq.getAbsolutePath()), Nil)
-
-  }
-
-  test("distribute local spark jars") {
-    val temp = Utils.createTempDir()
-    val jarsDir = new File(temp, "jars")
-    assert(jarsDir.mkdir())
-    val jar = TestUtils.createJarWithFiles(Map(), jarsDir)
-    new FileOutputStream(new File(temp, "RELEASE")).close()
-
-    val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> temp.getAbsolutePath()))
-    val client = createClient(sparkConf)
-    client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil)
-    classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*"))
-  }
-
-  test("ignore same name jars") {
-    val libs = Utils.createTempDir()
-    val jarsDir = new File(libs, "jars")
-    assert(jarsDir.mkdir())
-    new FileOutputStream(new File(libs, "RELEASE")).close()
-    val userLib1 = Utils.createTempDir()
-    val userLib2 = Utils.createTempDir()
-
-    val jar1 = TestUtils.createJarWithFiles(Map(), jarsDir)
-    val jar2 = TestUtils.createJarWithFiles(Map(), userLib1)
-    // Copy jar2 to jar3 with same name
-    val jar3 = {
-      val target = new File(userLib2, new File(jar2.toURI).getName)
-      val input = new FileInputStream(jar2.getPath)
-      val output = new FileOutputStream(target)
-      Utils.copyStream(input, output, closeStreams = true)
-      target.toURI.toURL
-    }
-
-    val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
-      .set(JARS_TO_DISTRIBUTE, Seq(jar2.getPath, jar3.getPath))
-
-    val client = createClient(sparkConf)
-    val tempDir = Utils.createTempDir()
-    client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil)
-
-    // Only jar2 will be added to SECONDARY_JARS, jar3 which has the same name with jar2 will be
-    // ignored.
-    sparkConf.get(SECONDARY_JARS) should be (Some(Seq(new File(jar2.toURI).getName)))
-  }
-
-  object Fixtures {
-
-    val knownDefYarnAppCP: Seq[String] =
-      getFieldValue[Array[String], Seq[String]](classOf[YarnConfiguration],
-                                                "DEFAULT_YARN_APPLICATION_CLASSPATH",
-                                                Seq[String]())(a => a.toSeq)
-
-
-    val knownDefMRAppCP: Seq[String] =
-      getFieldValue2[String, Array[String], Seq[String]](
-        classOf[MRJobConfig],
-        "DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH",
-        Seq[String]())(a => a.split(","))(a => a.toSeq)
-
-    val knownYARNAppCP = Some(Seq("/known/yarn/path"))
-
-    val knownMRAppCP = Some(Seq("/known/mr/path"))
-
-    val mapMRAppConf =
-      Map("mapreduce.application.classpath" -> knownMRAppCP.map(_.mkString(":")).get)
-
-    val mapYARNAppConf =
-      Map(YarnConfiguration.YARN_APPLICATION_CLASSPATH -> knownYARNAppCP.map(_.mkString(":")).get)
-
-    val mapAppConf = mapYARNAppConf ++ mapMRAppConf
-  }
-
-  def withAppConf(m: Map[String, String] = Map())(testCode: (Configuration) => Any) {
-    val conf = new Configuration
-    m.foreach { case (k, v) => conf.set(k, v, "ClientSpec") }
-    testCode(conf)
-  }
-
-  def newEnv: MutableHashMap[String, String] = MutableHashMap[String, String]()
-
-  def classpath(env: MutableHashMap[String, String]): Array[String] =
-    env(Environment.CLASSPATH.name).split(":|;|<CPS>")
-
-  def flatten(a: Option[Seq[String]], b: Option[Seq[String]]): Array[String] =
-    (a ++ b).flatten.toArray
-
-  def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B = {
-    Try(clazz.getField(field))
-      .map(_.get(null).asInstanceOf[A])
-      .toOption
-      .map(mapTo)
-      .getOrElse(defaults)
-  }
-
-  def getFieldValue2[A: ClassTag, A1: ClassTag, B](
-        clazz: Class[_],
-        field: String,
-        defaults: => B)(mapTo: A => B)(mapTo1: A1 => B): B = {
-    Try(clazz.getField(field)).map(_.get(null)).map {
-      case v: A => mapTo(v)
-      case v1: A1 => mapTo1(v1)
-      case _ => defaults
-    }.toOption.getOrElse(defaults)
-  }
-
-  private def createClient(
-      sparkConf: SparkConf,
-      conf: Configuration = new Configuration(),
-      args: Array[String] = Array()): Client = {
-    val clientArgs = new ClientArguments(args)
-    spy(new Client(clientArgs, conf, sparkConf))
-  }
-
-  private def classpath(client: Client): Array[String] = {
-    val env = new MutableHashMap[String, String]()
-    populateClasspath(null, client.hadoopConf, client.sparkConf, env)
-    classpath(env)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala
deleted file mode 100644
index afb4b69..0000000
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala
+++ /dev/null
@@ -1,153 +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.deploy.yarn
-
-import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
-import org.scalatest.{BeforeAndAfterEach, Matchers}
-
-import org.apache.spark.SparkFunSuite
-
-class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with BeforeAndAfterEach {
-
-  private val yarnAllocatorSuite = new YarnAllocatorSuite
-  import yarnAllocatorSuite._
-
-  def createContainerRequest(nodes: Array[String]): ContainerRequest =
-    new ContainerRequest(containerResource, nodes, null, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)
-
-  override def beforeEach() {
-    yarnAllocatorSuite.beforeEach()
-  }
-
-  override def afterEach() {
-    yarnAllocatorSuite.afterEach()
-  }
-
-  test("allocate locality preferred containers with enough resource and no matched existed " +
-    "containers") {
-    // 1. All the locations of current containers cannot satisfy the new requirements
-    // 2. Current requested container number can fully satisfy the pending tasks.
-
-    val handler = createAllocator(2)
-    handler.updateResourceRequests()
-    handler.handleAllocatedContainers(Array(createContainer("host1"), createContainer("host2")))
-
-    val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
-      3, 15, Map("host3" -> 15, "host4" -> 15, "host5" -> 10),
-        handler.allocatedHostToContainersMap, Seq.empty)
-
-    assert(localities.map(_.nodes) === Array(
-      Array("host3", "host4", "host5"),
-      Array("host3", "host4", "host5"),
-      Array("host3", "host4")))
-  }
-
-  test("allocate locality preferred containers with enough resource and partially matched " +
-    "containers") {
-    // 1. Parts of current containers' locations can satisfy the new requirements
-    // 2. Current requested container number can fully satisfy the pending tasks.
-
-    val handler = createAllocator(3)
-    handler.updateResourceRequests()
-    handler.handleAllocatedContainers(Array(
-      createContainer("host1"),
-      createContainer("host1"),
-      createContainer("host2")
-    ))
-
-    val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
-      3, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
-        handler.allocatedHostToContainersMap, Seq.empty)
-
-    assert(localities.map(_.nodes) ===
-      Array(null, Array("host2", "host3"), Array("host2", "host3")))
-  }
-
-  test("allocate locality preferred containers with limited resource and partially matched " +
-    "containers") {
-    // 1. Parts of current containers' locations can satisfy the new requirements
-    // 2. Current requested container number cannot fully satisfy the pending tasks.
-
-    val handler = createAllocator(3)
-    handler.updateResourceRequests()
-    handler.handleAllocatedContainers(Array(
-      createContainer("host1"),
-      createContainer("host1"),
-      createContainer("host2")
-    ))
-
-    val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
-      1, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
-        handler.allocatedHostToContainersMap, Seq.empty)
-
-    assert(localities.map(_.nodes) === Array(Array("host2", "host3")))
-  }
-
-  test("allocate locality preferred containers with fully matched containers") {
-    // Current containers' locations can fully satisfy the new requirements
-
-    val handler = createAllocator(5)
-    handler.updateResourceRequests()
-    handler.handleAllocatedContainers(Array(
-      createContainer("host1"),
-      createContainer("host1"),
-      createContainer("host2"),
-      createContainer("host2"),
-      createContainer("host3")
-    ))
-
-    val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
-      3, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
-        handler.allocatedHostToContainersMap, Seq.empty)
-
-    assert(localities.map(_.nodes) === Array(null, null, null))
-  }
-
-  test("allocate containers with no locality preference") {
-    // Request new container without locality preference
-
-    val handler = createAllocator(2)
-    handler.updateResourceRequests()
-    handler.handleAllocatedContainers(Array(createContainer("host1"), createContainer("host2")))
-
-    val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
-      1, 0, Map.empty, handler.allocatedHostToContainersMap, Seq.empty)
-
-    assert(localities.map(_.nodes) === Array(null))
-  }
-
-  test("allocate locality preferred containers by considering the localities of pending requests") {
-    val handler = createAllocator(3)
-    handler.updateResourceRequests()
-    handler.handleAllocatedContainers(Array(
-      createContainer("host1"),
-      createContainer("host1"),
-      createContainer("host2")
-    ))
-
-    val pendingAllocationRequests = Seq(
-      createContainerRequest(Array("host2", "host3")),
-      createContainerRequest(Array("host1", "host4")))
-
-    val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
-      1, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
-        handler.allocatedHostToContainersMap, pendingAllocationRequests)
-
-    assert(localities.map(_.nodes) === Array(Array("host3")))
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/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
deleted file mode 100644
index 994dc75..0000000
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala
+++ /dev/null
@@ -1,344 +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.deploy.yarn
-
-import java.util.{Arrays, List => JList}
-
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic
-import org.apache.hadoop.net.DNSToSwitchMapping
-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.mockito.Mockito._
-import org.scalatest.{BeforeAndAfterEach, Matchers}
-
-import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.deploy.yarn.YarnAllocator._
-import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._
-import org.apache.spark.rpc.RpcEndpointRef
-import org.apache.spark.scheduler.SplitInfo
-import org.apache.spark.util.ManualClock
-
-class MockResolver extends DNSToSwitchMapping {
-
-  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 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")
-  sparkConf.set(SPARK_JARS, Seq("notarealjar.jar"))
-  sparkConf.set("spark.yarn.launchContainers", "false")
-
-  val appAttemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 0), 0)
-
-  // Resource returned by YARN.  YARN can give larger containers than requested, so give 6 cores
-  // instead of the 5 requested and 3 GB instead of the 2 requested.
-  val containerResource = Resource.newInstance(3072, 6)
-
-  var rmClient: AMRMClient[ContainerRequest] = _
-
-  var containerNum = 0
-
-  override def beforeEach() {
-    super.beforeEach()
-    rmClient = AMRMClient.createAMRMClient()
-    rmClient.init(conf)
-    rmClient.start()
-  }
-
-  override def afterEach() {
-    try {
-      rmClient.stop()
-    } finally {
-      super.afterEach()
-    }
-  }
-
-  class MockSplitInfo(host: String) extends SplitInfo(null, host, null, 1, null) {
-    override def hashCode(): Int = 0
-    override def equals(other: Any): Boolean = false
-  }
-
-  def createAllocator(maxExecutors: Int = 5): YarnAllocator = {
-    val args = Array(
-      "--jar", "somejar.jar",
-      "--class", "SomeClass")
-    val sparkConfClone = sparkConf.clone()
-    sparkConfClone
-      .set("spark.executor.instances", maxExecutors.toString)
-      .set("spark.executor.cores", "5")
-      .set("spark.executor.memory", "2048")
-    new YarnAllocator(
-      "not used",
-      mock(classOf[RpcEndpointRef]),
-      conf,
-      sparkConfClone,
-      rmClient,
-      appAttemptId,
-      new SecurityManager(sparkConf),
-      Map())
-  }
-
-  def createContainer(host: String): Container = {
-    // When YARN 2.6+ is required, avoid deprecation by using version with long second arg
-    val containerId = ContainerId.newInstance(appAttemptId, containerNum)
-    containerNum += 1
-    val nodeId = NodeId.newInstance(host, 1000)
-    Container.newInstance(containerId, nodeId, "", containerResource, RM_REQUEST_PRIORITY, null)
-  }
-
-  test("single container allocated") {
-    // request a single container and receive it
-    val handler = createAllocator(1)
-    handler.updateResourceRequests()
-    handler.getNumExecutorsRunning should be (0)
-    handler.getPendingAllocate.size should be (1)
-
-    val container = createContainer("host1")
-    handler.handleAllocatedContainers(Array(container))
-
-    handler.getNumExecutorsRunning should be (1)
-    handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1")
-    handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId)
-
-    val size = rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size
-    size should be (0)
-  }
-
-  test("container should not be created if requested number if met") {
-    // request a single container and receive it
-    val handler = createAllocator(1)
-    handler.updateResourceRequests()
-    handler.getNumExecutorsRunning should be (0)
-    handler.getPendingAllocate.size should be (1)
-
-    val container = createContainer("host1")
-    handler.handleAllocatedContainers(Array(container))
-
-    handler.getNumExecutorsRunning should be (1)
-    handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1")
-    handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId)
-
-    val container2 = createContainer("host2")
-    handler.handleAllocatedContainers(Array(container2))
-    handler.getNumExecutorsRunning should be (1)
-  }
-
-  test("some containers allocated") {
-    // request a few containers and receive some of them
-    val handler = createAllocator(4)
-    handler.updateResourceRequests()
-    handler.getNumExecutorsRunning should be (0)
-    handler.getPendingAllocate.size should be (4)
-
-    val container1 = createContainer("host1")
-    val container2 = createContainer("host1")
-    val container3 = createContainer("host2")
-    handler.handleAllocatedContainers(Array(container1, container2, container3))
-
-    handler.getNumExecutorsRunning should be (3)
-    handler.allocatedContainerToHostMap.get(container1.getId).get should be ("host1")
-    handler.allocatedContainerToHostMap.get(container2.getId).get should be ("host1")
-    handler.allocatedContainerToHostMap.get(container3.getId).get should be ("host2")
-    handler.allocatedHostToContainersMap.get("host1").get should contain (container1.getId)
-    handler.allocatedHostToContainersMap.get("host1").get should contain (container2.getId)
-    handler.allocatedHostToContainersMap.get("host2").get should contain (container3.getId)
-  }
-
-  test("receive more containers than requested") {
-    val handler = createAllocator(2)
-    handler.updateResourceRequests()
-    handler.getNumExecutorsRunning should be (0)
-    handler.getPendingAllocate.size should be (2)
-
-    val container1 = createContainer("host1")
-    val container2 = createContainer("host2")
-    val container3 = createContainer("host4")
-    handler.handleAllocatedContainers(Array(container1, container2, container3))
-
-    handler.getNumExecutorsRunning should be (2)
-    handler.allocatedContainerToHostMap.get(container1.getId).get should be ("host1")
-    handler.allocatedContainerToHostMap.get(container2.getId).get should be ("host2")
-    handler.allocatedContainerToHostMap.contains(container3.getId) should be (false)
-    handler.allocatedHostToContainersMap.get("host1").get should contain (container1.getId)
-    handler.allocatedHostToContainersMap.get("host2").get should contain (container2.getId)
-    handler.allocatedHostToContainersMap.contains("host4") should be (false)
-  }
-
-  test("decrease total requested executors") {
-    val handler = createAllocator(4)
-    handler.updateResourceRequests()
-    handler.getNumExecutorsRunning should be (0)
-    handler.getPendingAllocate.size should be (4)
-
-    handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty)
-    handler.updateResourceRequests()
-    handler.getPendingAllocate.size should be (3)
-
-    val container = createContainer("host1")
-    handler.handleAllocatedContainers(Array(container))
-
-    handler.getNumExecutorsRunning should be (1)
-    handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1")
-    handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId)
-
-    handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map.empty)
-    handler.updateResourceRequests()
-    handler.getPendingAllocate.size should be (1)
-  }
-
-  test("decrease total requested executors to less than currently running") {
-    val handler = createAllocator(4)
-    handler.updateResourceRequests()
-    handler.getNumExecutorsRunning should be (0)
-    handler.getPendingAllocate.size should be (4)
-
-    handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty)
-    handler.updateResourceRequests()
-    handler.getPendingAllocate.size should be (3)
-
-    val container1 = createContainer("host1")
-    val container2 = createContainer("host2")
-    handler.handleAllocatedContainers(Array(container1, container2))
-
-    handler.getNumExecutorsRunning should be (2)
-
-    handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty)
-    handler.updateResourceRequests()
-    handler.getPendingAllocate.size should be (0)
-    handler.getNumExecutorsRunning should be (2)
-  }
-
-  test("kill executors") {
-    val handler = createAllocator(4)
-    handler.updateResourceRequests()
-    handler.getNumExecutorsRunning should be (0)
-    handler.getPendingAllocate.size should be (4)
-
-    val container1 = createContainer("host1")
-    val container2 = createContainer("host2")
-    handler.handleAllocatedContainers(Array(container1, container2))
-
-    handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty)
-    handler.executorIdToContainer.keys.foreach { id => handler.killExecutor(id ) }
-
-    val statuses = Seq(container1, container2).map { c =>
-      ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Finished", 0)
-    }
-    handler.updateResourceRequests()
-    handler.processCompletedContainers(statuses.toSeq)
-    handler.getNumExecutorsRunning should be (0)
-    handler.getPendingAllocate.size should be (1)
-  }
-
-  test("lost executor removed from backend") {
-    val handler = createAllocator(4)
-    handler.updateResourceRequests()
-    handler.getNumExecutorsRunning should be (0)
-    handler.getPendingAllocate.size should be (4)
-
-    val container1 = createContainer("host1")
-    val container2 = createContainer("host2")
-    handler.handleAllocatedContainers(Array(container1, container2))
-
-    handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map())
-
-    val statuses = Seq(container1, container2).map { c =>
-      ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Failed", -1)
-    }
-    handler.updateResourceRequests()
-    handler.processCompletedContainers(statuses.toSeq)
-    handler.updateResourceRequests()
-    handler.getNumExecutorsRunning should be (0)
-    handler.getPendingAllocate.size should be (2)
-    handler.getNumExecutorsFailed should be (2)
-    handler.getNumUnexpectedContainerRelease should be (2)
-  }
-
-  test("memory exceeded diagnostic regexes") {
-    val diagnostics =
-      "Container [pid=12465,containerID=container_1412887393566_0003_01_000002] is running " +
-        "beyond physical memory limits. Current usage: 2.1 MB of 2 GB physical memory used; " +
-        "5.8 GB of 4.2 GB virtual memory used. Killing container."
-    val vmemMsg = memLimitExceededLogMessage(diagnostics, VMEM_EXCEEDED_PATTERN)
-    val pmemMsg = memLimitExceededLogMessage(diagnostics, PMEM_EXCEEDED_PATTERN)
-    assert(vmemMsg.contains("5.8 GB of 4.2 GB virtual memory used."))
-    assert(pmemMsg.contains("2.1 MB of 2 GB physical memory used."))
-  }
-
-  test("window based failure executor counting") {
-    sparkConf.set("spark.yarn.executor.failuresValidityInterval", "100s")
-    val handler = createAllocator(4)
-    val clock = new ManualClock(0L)
-    handler.setClock(clock)
-
-    handler.updateResourceRequests()
-    handler.getNumExecutorsRunning should be (0)
-    handler.getPendingAllocate.size should be (4)
-
-    val containers = Seq(
-      createContainer("host1"),
-      createContainer("host2"),
-      createContainer("host3"),
-      createContainer("host4")
-    )
-    handler.handleAllocatedContainers(containers)
-
-    val failedStatuses = containers.map { c =>
-      ContainerStatus.newInstance(c.getId, ContainerState.COMPLETE, "Failed", -1)
-    }
-
-    handler.getNumExecutorsFailed should be (0)
-
-    clock.advance(100 * 1000L)
-    handler.processCompletedContainers(failedStatuses.slice(0, 1))
-    handler.getNumExecutorsFailed should be (1)
-
-    clock.advance(101 * 1000L)
-    handler.getNumExecutorsFailed should be (0)
-
-    handler.processCompletedContainers(failedStatuses.slice(1, 3))
-    handler.getNumExecutorsFailed should be (2)
-
-    clock.advance(50 * 1000L)
-    handler.processCompletedContainers(failedStatuses.slice(3, 4))
-    handler.getNumExecutorsFailed should be (3)
-
-    clock.advance(51 * 1000L)
-    handler.getNumExecutorsFailed should be (1)
-
-    clock.advance(50 * 1000L)
-    handler.getNumExecutorsFailed should be (0)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
deleted file mode 100644
index 99fb58a..0000000
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ /dev/null
@@ -1,493 +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.deploy.yarn
-
-import java.io.File
-import java.net.URL
-import java.nio.charset.StandardCharsets
-import java.util.{HashMap => JHashMap}
-
-import scala.collection.mutable
-import scala.concurrent.duration._
-import scala.language.postfixOps
-
-import com.google.common.io.{ByteStreams, Files}
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.scalatest.Matchers
-import org.scalatest.concurrent.Eventually._
-
-import org.apache.spark._
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.internal.Logging
-import org.apache.spark.launcher._
-import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart,
-  SparkListenerExecutorAdded}
-import org.apache.spark.scheduler.cluster.ExecutorInfo
-import org.apache.spark.tags.ExtendedYarnTest
-import org.apache.spark.util.Utils
-
-/**
- * Integration tests for YARN; these tests use a mini Yarn cluster to run Spark-on-YARN
- * applications, and require the Spark assembly to be built before they can be successfully
- * run.
- */
-@ExtendedYarnTest
-class YarnClusterSuite extends BaseYarnClusterSuite {
-
-  override def newYarnConfig(): YarnConfiguration = new YarnConfiguration()
-
-  private val TEST_PYFILE = """
-    |import mod1, mod2
-    |import sys
-    |from operator import add
-    |
-    |from pyspark import SparkConf , SparkContext
-    |if __name__ == "__main__":
-    |    if len(sys.argv) != 2:
-    |        print >> sys.stderr, "Usage: test.py [result file]"
-    |        exit(-1)
-    |    sc = SparkContext(conf=SparkConf())
-    |    status = open(sys.argv[1],'w')
-    |    result = "failure"
-    |    rdd = sc.parallelize(range(10)).map(lambda x: x * mod1.func() * mod2.func())
-    |    cnt = rdd.count()
-    |    if cnt == 10:
-    |        result = "success"
-    |    status.write(result)
-    |    status.close()
-    |    sc.stop()
-    """.stripMargin
-
-  private val TEST_PYMODULE = """
-    |def func():
-    |    return 42
-    """.stripMargin
-
-  test("run Spark in yarn-client mode") {
-    testBasicYarnApp(true)
-  }
-
-  test("run Spark in yarn-cluster mode") {
-    testBasicYarnApp(false)
-  }
-
-  test("run Spark in yarn-client mode with different configurations") {
-    testBasicYarnApp(true,
-      Map(
-        "spark.driver.memory" -> "512m",
-        "spark.executor.cores" -> "1",
-        "spark.executor.memory" -> "512m",
-        "spark.executor.instances" -> "2"
-      ))
-  }
-
-  test("run Spark in yarn-cluster mode with different configurations") {
-    testBasicYarnApp(false,
-      Map(
-        "spark.driver.memory" -> "512m",
-        "spark.driver.cores" -> "1",
-        "spark.executor.cores" -> "1",
-        "spark.executor.memory" -> "512m",
-        "spark.executor.instances" -> "2"
-      ))
-  }
-
-  test("run Spark in yarn-cluster mode with using SparkHadoopUtil.conf") {
-    testYarnAppUseSparkHadoopUtilConf()
-  }
-
-  test("run Spark in yarn-client mode with additional jar") {
-    testWithAddJar(true)
-  }
-
-  test("run Spark in yarn-cluster mode with additional jar") {
-    testWithAddJar(false)
-  }
-
-  test("run Spark in yarn-cluster mode unsuccessfully") {
-    // Don't provide arguments so the driver will fail.
-    val finalState = runSpark(false, mainClassName(YarnClusterDriver.getClass))
-    finalState should be (SparkAppHandle.State.FAILED)
-  }
-
-  test("run Spark in yarn-cluster mode failure after sc initialized") {
-    val finalState = runSpark(false, mainClassName(YarnClusterDriverWithFailure.getClass))
-    finalState should be (SparkAppHandle.State.FAILED)
-  }
-
-  test("run Python application in yarn-client mode") {
-    testPySpark(true)
-  }
-
-  test("run Python application in yarn-cluster mode") {
-    testPySpark(false)
-  }
-
-  test("run Python application in yarn-cluster mode using " +
-    " spark.yarn.appMasterEnv to override local envvar") {
-    testPySpark(
-      clientMode = false,
-      extraConf = Map(
-        "spark.yarn.appMasterEnv.PYSPARK_DRIVER_PYTHON"
-          -> sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python"),
-        "spark.yarn.appMasterEnv.PYSPARK_PYTHON"
-          -> sys.env.getOrElse("PYSPARK_PYTHON", "python")),
-      extraEnv = Map(
-        "PYSPARK_DRIVER_PYTHON" -> "not python",
-        "PYSPARK_PYTHON" -> "not python"))
-  }
-
-  test("user class path first in client mode") {
-    testUseClassPathFirst(true)
-  }
-
-  test("user class path first in cluster mode") {
-    testUseClassPathFirst(false)
-  }
-
-  test("monitor app using launcher library") {
-    val env = new JHashMap[String, String]()
-    env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
-
-    val propsFile = createConfFile()
-    val handle = new SparkLauncher(env)
-      .setSparkHome(sys.props("spark.test.home"))
-      .setConf("spark.ui.enabled", "false")
-      .setPropertiesFile(propsFile)
-      .setMaster("yarn")
-      .setDeployMode("client")
-      .setAppResource(SparkLauncher.NO_RESOURCE)
-      .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
-      .startApplication()
-
-    try {
-      eventually(timeout(30 seconds), interval(100 millis)) {
-        handle.getState() should be (SparkAppHandle.State.RUNNING)
-      }
-
-      handle.getAppId() should not be (null)
-      handle.getAppId() should startWith ("application_")
-      handle.stop()
-
-      eventually(timeout(30 seconds), interval(100 millis)) {
-        handle.getState() should be (SparkAppHandle.State.KILLED)
-      }
-    } finally {
-      handle.kill()
-    }
-  }
-
-  test("timeout to get SparkContext in cluster mode triggers failure") {
-    val timeout = 2000
-    val finalState = runSpark(false, mainClassName(SparkContextTimeoutApp.getClass),
-      appArgs = Seq((timeout * 4).toString),
-      extraConf = Map(AM_MAX_WAIT_TIME.key -> timeout.toString))
-    finalState should be (SparkAppHandle.State.FAILED)
-  }
-
-  private def testBasicYarnApp(clientMode: Boolean, conf: Map[String, String] = Map()): Unit = {
-    val result = File.createTempFile("result", null, tempDir)
-    val finalState = runSpark(clientMode, mainClassName(YarnClusterDriver.getClass),
-      appArgs = Seq(result.getAbsolutePath()),
-      extraConf = conf)
-    checkResult(finalState, result)
-  }
-
-  private def testYarnAppUseSparkHadoopUtilConf(): Unit = {
-    val result = File.createTempFile("result", null, tempDir)
-    val finalState = runSpark(false,
-      mainClassName(YarnClusterDriverUseSparkHadoopUtilConf.getClass),
-      appArgs = Seq("key=value", result.getAbsolutePath()),
-      extraConf = Map("spark.hadoop.key" -> "value"))
-    checkResult(finalState, result)
-  }
-
-  private def testWithAddJar(clientMode: Boolean): Unit = {
-    val originalJar = TestUtils.createJarWithFiles(Map("test.resource" -> "ORIGINAL"), tempDir)
-    val driverResult = File.createTempFile("driver", null, tempDir)
-    val executorResult = File.createTempFile("executor", null, tempDir)
-    val finalState = runSpark(clientMode, mainClassName(YarnClasspathTest.getClass),
-      appArgs = Seq(driverResult.getAbsolutePath(), executorResult.getAbsolutePath()),
-      extraClassPath = Seq(originalJar.getPath()),
-      extraJars = Seq("local:" + originalJar.getPath()))
-    checkResult(finalState, driverResult, "ORIGINAL")
-    checkResult(finalState, executorResult, "ORIGINAL")
-  }
-
-  private def testPySpark(
-      clientMode: Boolean,
-      extraConf: Map[String, String] = Map(),
-      extraEnv: Map[String, String] = Map()): Unit = {
-    val primaryPyFile = new File(tempDir, "test.py")
-    Files.write(TEST_PYFILE, primaryPyFile, StandardCharsets.UTF_8)
-
-    // When running tests, let's not assume the user has built the assembly module, which also
-    // creates the pyspark archive. Instead, let's use PYSPARK_ARCHIVES_PATH to point at the
-    // needed locations.
-    val sparkHome = sys.props("spark.test.home")
-    val pythonPath = Seq(
-        s"$sparkHome/python/lib/py4j-0.10.4-src.zip",
-        s"$sparkHome/python")
-    val extraEnvVars = Map(
-      "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator),
-      "PYTHONPATH" -> pythonPath.mkString(File.pathSeparator)) ++ extraEnv
-
-    val moduleDir =
-      if (clientMode) {
-        // In client-mode, .py files added with --py-files are not visible in the driver.
-        // This is something that the launcher library would have to handle.
-        tempDir
-      } else {
-        val subdir = new File(tempDir, "pyModules")
-        subdir.mkdir()
-        subdir
-      }
-    val pyModule = new File(moduleDir, "mod1.py")
-    Files.write(TEST_PYMODULE, pyModule, StandardCharsets.UTF_8)
-
-    val mod2Archive = TestUtils.createJarWithFiles(Map("mod2.py" -> TEST_PYMODULE), moduleDir)
-    val pyFiles = Seq(pyModule.getAbsolutePath(), mod2Archive.getPath()).mkString(",")
-    val result = File.createTempFile("result", null, tempDir)
-
-    val finalState = runSpark(clientMode, primaryPyFile.getAbsolutePath(),
-      sparkArgs = Seq("--py-files" -> pyFiles),
-      appArgs = Seq(result.getAbsolutePath()),
-      extraEnv = extraEnvVars,
-      extraConf = extraConf)
-    checkResult(finalState, result)
-  }
-
-  private def testUseClassPathFirst(clientMode: Boolean): Unit = {
-    // Create a jar file that contains a different version of "test.resource".
-    val originalJar = TestUtils.createJarWithFiles(Map("test.resource" -> "ORIGINAL"), tempDir)
-    val userJar = TestUtils.createJarWithFiles(Map("test.resource" -> "OVERRIDDEN"), tempDir)
-    val driverResult = File.createTempFile("driver", null, tempDir)
-    val executorResult = File.createTempFile("executor", null, tempDir)
-    val finalState = runSpark(clientMode, mainClassName(YarnClasspathTest.getClass),
-      appArgs = Seq(driverResult.getAbsolutePath(), executorResult.getAbsolutePath()),
-      extraClassPath = Seq(originalJar.getPath()),
-      extraJars = Seq("local:" + userJar.getPath()),
-      extraConf = Map(
-        "spark.driver.userClassPathFirst" -> "true",
-        "spark.executor.userClassPathFirst" -> "true"))
-    checkResult(finalState, driverResult, "OVERRIDDEN")
-    checkResult(finalState, executorResult, "OVERRIDDEN")
-  }
-
-}
-
-private[spark] class SaveExecutorInfo extends SparkListener {
-  val addedExecutorInfos = mutable.Map[String, ExecutorInfo]()
-  var driverLogs: Option[collection.Map[String, String]] = None
-
-  override def onExecutorAdded(executor: SparkListenerExecutorAdded) {
-    addedExecutorInfos(executor.executorId) = executor.executorInfo
-  }
-
-  override def onApplicationStart(appStart: SparkListenerApplicationStart): Unit = {
-    driverLogs = appStart.driverLogs
-  }
-}
-
-private object YarnClusterDriverWithFailure extends Logging with Matchers {
-  def main(args: Array[String]): Unit = {
-    val sc = new SparkContext(new SparkConf()
-      .set("spark.extraListeners", classOf[SaveExecutorInfo].getName)
-      .setAppName("yarn test with failure"))
-
-    throw new Exception("exception after sc initialized")
-  }
-}
-
-private object YarnClusterDriverUseSparkHadoopUtilConf extends Logging with Matchers {
-  def main(args: Array[String]): Unit = {
-    if (args.length != 2) {
-      // scalastyle:off println
-      System.err.println(
-        s"""
-        |Invalid command line: ${args.mkString(" ")}
-        |
-        |Usage: YarnClusterDriverUseSparkHadoopUtilConf [hadoopConfKey=value] [result file]
-        """.stripMargin)
-      // scalastyle:on println
-      System.exit(1)
-    }
-
-    val sc = new SparkContext(new SparkConf()
-      .set("spark.extraListeners", classOf[SaveExecutorInfo].getName)
-      .setAppName("yarn test using SparkHadoopUtil's conf"))
-
-    val kv = args(0).split("=")
-    val status = new File(args(1))
-    var result = "failure"
-    try {
-      SparkHadoopUtil.get.conf.get(kv(0)) should be (kv(1))
-      result = "success"
-    } finally {
-      Files.write(result, status, StandardCharsets.UTF_8)
-      sc.stop()
-    }
-  }
-}
-
-private object YarnClusterDriver extends Logging with Matchers {
-
-  val WAIT_TIMEOUT_MILLIS = 10000
-
-  def main(args: Array[String]): Unit = {
-    if (args.length != 1) {
-      // scalastyle:off println
-      System.err.println(
-        s"""
-        |Invalid command line: ${args.mkString(" ")}
-        |
-        |Usage: YarnClusterDriver [result file]
-        """.stripMargin)
-      // scalastyle:on println
-      System.exit(1)
-    }
-
-    val sc = new SparkContext(new SparkConf()
-      .set("spark.extraListeners", classOf[SaveExecutorInfo].getName)
-      .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns"))
-    val conf = sc.getConf
-    val status = new File(args(0))
-    var result = "failure"
-    try {
-      val data = sc.parallelize(1 to 4, 4).collect().toSet
-      sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
-      data should be (Set(1, 2, 3, 4))
-      result = "success"
-
-      // Verify that the config archive is correctly placed in the classpath of all containers.
-      val confFile = "/" + Client.SPARK_CONF_FILE
-      assert(getClass().getResource(confFile) != null)
-      val configFromExecutors = sc.parallelize(1 to 4, 4)
-        .map { _ => Option(getClass().getResource(confFile)).map(_.toString).orNull }
-        .collect()
-      assert(configFromExecutors.find(_ == null) === None)
-    } finally {
-      Files.write(result, status, StandardCharsets.UTF_8)
-      sc.stop()
-    }
-
-    // verify log urls are present
-    val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo]
-    assert(listeners.size === 1)
-    val listener = listeners(0)
-    val executorInfos = listener.addedExecutorInfos.values
-    assert(executorInfos.nonEmpty)
-    executorInfos.foreach { info =>
-      assert(info.logUrlMap.nonEmpty)
-    }
-
-    // If we are running in yarn-cluster mode, verify that driver logs links and present and are
-    // in the expected format.
-    if (conf.get("spark.submit.deployMode") == "cluster") {
-      assert(listener.driverLogs.nonEmpty)
-      val driverLogs = listener.driverLogs.get
-      assert(driverLogs.size === 2)
-      assert(driverLogs.contains("stderr"))
-      assert(driverLogs.contains("stdout"))
-      val urlStr = driverLogs("stderr")
-      // Ensure that this is a valid URL, else this will throw an exception
-      new URL(urlStr)
-      val containerId = YarnSparkHadoopUtil.get.getContainerId
-      val user = Utils.getCurrentUserName()
-      assert(urlStr.endsWith(s"/node/containerlogs/$containerId/$user/stderr?start=-4096"))
-    }
-  }
-
-}
-
-private object YarnClasspathTest extends Logging {
-  def error(m: String, ex: Throwable = null): Unit = {
-    logError(m, ex)
-    // scalastyle:off println
-    System.out.println(m)
-    if (ex != null) {
-      ex.printStackTrace(System.out)
-    }
-    // scalastyle:on println
-  }
-
-  def main(args: Array[String]): Unit = {
-    if (args.length != 2) {
-      error(
-        s"""
-        |Invalid command line: ${args.mkString(" ")}
-        |
-        |Usage: YarnClasspathTest [driver result file] [executor result file]
-        """.stripMargin)
-      // scalastyle:on println
-    }
-
-    readResource(args(0))
-    val sc = new SparkContext(new SparkConf())
-    try {
-      sc.parallelize(Seq(1)).foreach { x => readResource(args(1)) }
-    } finally {
-      sc.stop()
-    }
-  }
-
-  private def readResource(resultPath: String): Unit = {
-    var result = "failure"
-    try {
-      val ccl = Thread.currentThread().getContextClassLoader()
-      val resource = ccl.getResourceAsStream("test.resource")
-      val bytes = ByteStreams.toByteArray(resource)
-      result = new String(bytes, 0, bytes.length, StandardCharsets.UTF_8)
-    } catch {
-      case t: Throwable =>
-        error(s"loading test.resource to $resultPath", t)
-    } finally {
-      Files.write(result, new File(resultPath), StandardCharsets.UTF_8)
-    }
-  }
-
-}
-
-private object YarnLauncherTestApp {
-
-  def main(args: Array[String]): Unit = {
-    // Do not stop the application; the test will stop it using the launcher lib. Just run a task
-    // that will prevent the process from exiting.
-    val sc = new SparkContext(new SparkConf())
-    sc.parallelize(Seq(1)).foreach { i =>
-      this.synchronized {
-        wait()
-      }
-    }
-  }
-
-}
-
-/**
- * Used to test code in the AM that detects the SparkContext instance. Expects a single argument
- * with the duration to sleep for, in ms.
- */
-private object SparkContextTimeoutApp {
-
-  def main(args: Array[String]): Unit = {
-    val Array(sleepTime) = args
-    Thread.sleep(java.lang.Long.parseLong(sleepTime))
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala
deleted file mode 100644
index 950ebd9..0000000
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala
+++ /dev/null
@@ -1,112 +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.deploy.yarn
-
-import java.io.File
-import java.nio.charset.StandardCharsets
-
-import com.google.common.io.Files
-import org.apache.commons.io.FileUtils
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.scalatest.Matchers
-
-import org.apache.spark._
-import org.apache.spark.internal.Logging
-import org.apache.spark.network.shuffle.ShuffleTestAccessor
-import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor}
-import org.apache.spark.tags.ExtendedYarnTest
-
-/**
- * Integration test for the external shuffle service with a yarn mini-cluster
- */
-@ExtendedYarnTest
-class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite {
-
-  override def newYarnConfig(): YarnConfiguration = {
-    val yarnConfig = new YarnConfiguration()
-    yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle")
-    yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"),
-      classOf[YarnShuffleService].getCanonicalName)
-    yarnConfig.set("spark.shuffle.service.port", "0")
-    yarnConfig
-  }
-
-  test("external shuffle service") {
-    val shuffleServicePort = YarnTestAccessor.getShuffleServicePort
-    val shuffleService = YarnTestAccessor.getShuffleServiceInstance
-
-    val registeredExecFile = YarnTestAccessor.getRegisteredExecutorFile(shuffleService)
-
-    logInfo("Shuffle service port = " + shuffleServicePort)
-    val result = File.createTempFile("result", null, tempDir)
-    val finalState = runSpark(
-      false,
-      mainClassName(YarnExternalShuffleDriver.getClass),
-      appArgs = Seq(result.getAbsolutePath(), registeredExecFile.getAbsolutePath),
-      extraConf = Map(
-        "spark.shuffle.service.enabled" -> "true",
-        "spark.shuffle.service.port" -> shuffleServicePort.toString
-      )
-    )
-    checkResult(finalState, result)
-    assert(YarnTestAccessor.getRegisteredExecutorFile(shuffleService).exists())
-  }
-}
-
-private object YarnExternalShuffleDriver extends Logging with Matchers {
-
-  val WAIT_TIMEOUT_MILLIS = 10000
-
-  def main(args: Array[String]): Unit = {
-    if (args.length != 2) {
-      // scalastyle:off println
-      System.err.println(
-        s"""
-        |Invalid command line: ${args.mkString(" ")}
-        |
-        |Usage: ExternalShuffleDriver [result file] [registered exec file]
-        """.stripMargin)
-      // scalastyle:on println
-      System.exit(1)
-    }
-
-    val sc = new SparkContext(new SparkConf()
-      .setAppName("External Shuffle Test"))
-    val conf = sc.getConf
-    val status = new File(args(0))
-    val registeredExecFile = new File(args(1))
-    logInfo("shuffle service executor file = " + registeredExecFile)
-    var result = "failure"
-    val execStateCopy = new File(registeredExecFile.getAbsolutePath + "_dup")
-    try {
-      val data = sc.parallelize(0 until 100, 10).map { x => (x % 10) -> x }.reduceByKey{ _ + _ }.
-        collect().toSet
-      sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
-      data should be ((0 until 10).map{x => x -> (x * 10 + 450)}.toSet)
-      result = "success"
-      // only one process can open a leveldb file at a time, so we copy the files
-      FileUtils.copyDirectory(registeredExecFile, execStateCopy)
-      assert(!ShuffleTestAccessor.reloadRegisteredExecutors(execStateCopy).isEmpty)
-    } finally {
-      sc.stop()
-      FileUtils.deleteDirectory(execStateCopy)
-      Files.write(result, status, StandardCharsets.UTF_8)
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
deleted file mode 100644
index 7fbbe12..0000000
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
+++ /dev/null
@@ -1,213 +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.deploy.yarn
-
-import java.io.{File, IOException}
-import java.nio.charset.StandardCharsets
-
-import com.google.common.io.{ByteStreams, Files}
-import org.apache.hadoop.io.Text
-import org.apache.hadoop.yarn.api.ApplicationConstants
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.scalatest.Matchers
-
-import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.internal.Logging
-import org.apache.spark.util.{ResetSystemProperties, Utils}
-
-class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging
-  with ResetSystemProperties {
-
-  val hasBash =
-    try {
-      val exitCode = Runtime.getRuntime().exec(Array("bash", "--version")).waitFor()
-      exitCode == 0
-    } catch {
-      case e: IOException =>
-        false
-    }
-
-  if (!hasBash) {
-    logWarning("Cannot execute bash, skipping bash tests.")
-  }
-
-  def bashTest(name: String)(fn: => Unit): Unit =
-    if (hasBash) test(name)(fn) else ignore(name)(fn)
-
-  bashTest("shell script escaping") {
-    val scriptFile = File.createTempFile("script.", ".sh", Utils.createTempDir())
-    val args = Array("arg1", "${arg.2}", "\"arg3\"", "'arg4'", "$arg5", "\\arg6")
-    try {
-      val argLine = args.map(a => YarnSparkHadoopUtil.escapeForShell(a)).mkString(" ")
-      Files.write(("bash -c \"echo " + argLine + "\"").getBytes(StandardCharsets.UTF_8), scriptFile)
-      scriptFile.setExecutable(true)
-
-      val proc = Runtime.getRuntime().exec(Array(scriptFile.getAbsolutePath()))
-      val out = new String(ByteStreams.toByteArray(proc.getInputStream())).trim()
-      val err = new String(ByteStreams.toByteArray(proc.getErrorStream()))
-      val exitCode = proc.waitFor()
-      exitCode should be (0)
-      out should be (args.mkString(" "))
-    } finally {
-      scriptFile.delete()
-    }
-  }
-
-  test("Yarn configuration override") {
-    val key = "yarn.nodemanager.hostname"
-    val default = new YarnConfiguration()
-
-    val sparkConf = new SparkConf()
-      .set("spark.hadoop." + key, "someHostName")
-    val yarnConf = new YarnSparkHadoopUtil().newConfiguration(sparkConf)
-
-    yarnConf.getClass() should be (classOf[YarnConfiguration])
-    yarnConf.get(key) should not be default.get(key)
-  }
-
-
-  test("test getApplicationAclsForYarn acls on") {
-
-    // spark acls on, just pick up default user
-    val sparkConf = new SparkConf()
-    sparkConf.set("spark.acls.enable", "true")
-
-    val securityMgr = new SecurityManager(sparkConf)
-    val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)
-
-    val viewAcls = acls.get(ApplicationAccessType.VIEW_APP)
-    val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP)
-
-    viewAcls match {
-      case Some(vacls) =>
-        val aclSet = vacls.split(',').map(_.trim).toSet
-        assert(aclSet.contains(System.getProperty("user.name", "invalid")))
-      case None =>
-        fail()
-    }
-    modifyAcls match {
-      case Some(macls) =>
-        val aclSet = macls.split(',').map(_.trim).toSet
-        assert(aclSet.contains(System.getProperty("user.name", "invalid")))
-      case None =>
-        fail()
-    }
-  }
-
-  test("test getApplicationAclsForYarn acls on and specify users") {
-
-    // default spark acls are on and specify acls
-    val sparkConf = new SparkConf()
-    sparkConf.set("spark.acls.enable", "true")
-    sparkConf.set("spark.ui.view.acls", "user1,user2")
-    sparkConf.set("spark.modify.acls", "user3,user4")
-
-    val securityMgr = new SecurityManager(sparkConf)
-    val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)
-
-    val viewAcls = acls.get(ApplicationAccessType.VIEW_APP)
-    val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP)
-
-    viewAcls match {
-      case Some(vacls) =>
-        val aclSet = vacls.split(',').map(_.trim).toSet
-        assert(aclSet.contains("user1"))
-        assert(aclSet.contains("user2"))
-        assert(aclSet.contains(System.getProperty("user.name", "invalid")))
-      case None =>
-        fail()
-    }
-    modifyAcls match {
-      case Some(macls) =>
-        val aclSet = macls.split(',').map(_.trim).toSet
-        assert(aclSet.contains("user3"))
-        assert(aclSet.contains("user4"))
-        assert(aclSet.contains(System.getProperty("user.name", "invalid")))
-      case None =>
-        fail()
-    }
-
-  }
-
-  test("test expandEnvironment result") {
-    val target = Environment.PWD
-    if (classOf[Environment].getMethods().exists(_.getName == "$$")) {
-      YarnSparkHadoopUtil.expandEnvironment(target) should be ("{{" + target + "}}")
-    } else if (Utils.isWindows) {
-      YarnSparkHadoopUtil.expandEnvironment(target) should be ("%" + target + "%")
-    } else {
-      YarnSparkHadoopUtil.expandEnvironment(target) should be ("$" + target)
-    }
-
-  }
-
-  test("test getClassPathSeparator result") {
-    if (classOf[ApplicationConstants].getFields().exists(_.getName == "CLASS_PATH_SEPARATOR")) {
-      YarnSparkHadoopUtil.getClassPathSeparator() should be ("<CPS>")
-    } else if (Utils.isWindows) {
-      YarnSparkHadoopUtil.getClassPathSeparator() should be (";")
-    } else {
-      YarnSparkHadoopUtil.getClassPathSeparator() should be (":")
-    }
-  }
-
-  test("check different hadoop utils based on env variable") {
-    try {
-      System.setProperty("SPARK_YARN_MODE", "true")
-      assert(SparkHadoopUtil.get.getClass === classOf[YarnSparkHadoopUtil])
-      System.setProperty("SPARK_YARN_MODE", "false")
-      assert(SparkHadoopUtil.get.getClass === classOf[SparkHadoopUtil])
-    } finally {
-      System.clearProperty("SPARK_YARN_MODE")
-    }
-  }
-
-
-
-  // This test needs to live here because it depends on isYarnMode returning true, which can only
-  // happen in the YARN module.
-  test("security manager token generation") {
-    try {
-      System.setProperty("SPARK_YARN_MODE", "true")
-      val initial = SparkHadoopUtil.get
-        .getSecretKeyFromUserCredentials(SecurityManager.SECRET_LOOKUP_KEY)
-      assert(initial === null || initial.length === 0)
-
-      val conf = new SparkConf()
-        .set(SecurityManager.SPARK_AUTH_CONF, "true")
-        .set(SecurityManager.SPARK_AUTH_SECRET_CONF, "unused")
-      val sm = new SecurityManager(conf)
-
-      val generated = SparkHadoopUtil.get
-        .getSecretKeyFromUserCredentials(SecurityManager.SECRET_LOOKUP_KEY)
-      assert(generated != null)
-      val genString = new Text(generated).toString()
-      assert(genString != "unused")
-      assert(sm.getSecretKey() === genString)
-    } finally {
-      // removeSecretKey() was only added in Hadoop 2.6, so instead we just set the secret
-      // to an empty string.
-      SparkHadoopUtil.get.addSecretKeyToUserCredentials(SecurityManager.SECRET_LOOKUP_KEY, "")
-      System.clearProperty("SPARK_YARN_MODE")
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala
deleted file mode 100644
index db4619e..0000000
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala
+++ /dev/null
@@ -1,150 +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.deploy.yarn.security
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.io.Text
-import org.apache.hadoop.security.Credentials
-import org.apache.hadoop.security.token.Token
-import org.scalatest.{BeforeAndAfter, Matchers}
-
-import org.apache.spark.{SparkConf, SparkFunSuite}
-import org.apache.spark.deploy.yarn.config._
-
-class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfter {
-  private var credentialManager: ConfigurableCredentialManager = null
-  private var sparkConf: SparkConf = null
-  private var hadoopConf: Configuration = null
-
-  override def beforeAll(): Unit = {
-    super.beforeAll()
-
-    sparkConf = new SparkConf()
-    hadoopConf = new Configuration()
-    System.setProperty("SPARK_YARN_MODE", "true")
-  }
-
-  override def afterAll(): Unit = {
-    System.clearProperty("SPARK_YARN_MODE")
-
-    super.afterAll()
-  }
-
-  test("Correctly load default credential providers") {
-    credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
-
-    credentialManager.getServiceCredentialProvider("hdfs") should not be (None)
-    credentialManager.getServiceCredentialProvider("hbase") should not be (None)
-    credentialManager.getServiceCredentialProvider("hive") should not be (None)
-  }
-
-  test("disable hive credential provider") {
-    sparkConf.set("spark.yarn.security.credentials.hive.enabled", "false")
-    credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
-
-    credentialManager.getServiceCredentialProvider("hdfs") should not be (None)
-    credentialManager.getServiceCredentialProvider("hbase") should not be (None)
-    credentialManager.getServiceCredentialProvider("hive") should be (None)
-  }
-
-  test("using deprecated configurations") {
-    sparkConf.set("spark.yarn.security.tokens.hdfs.enabled", "false")
-    sparkConf.set("spark.yarn.security.tokens.hive.enabled", "false")
-    credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
-
-    credentialManager.getServiceCredentialProvider("hdfs") should be (None)
-    credentialManager.getServiceCredentialProvider("hive") should be (None)
-    credentialManager.getServiceCredentialProvider("test") should not be (None)
-    credentialManager.getServiceCredentialProvider("hbase") should not be (None)
-  }
-
-  test("verify obtaining credentials from provider") {
-    credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
-    val creds = new Credentials()
-
-    // Tokens can only be obtained from TestTokenProvider, for hdfs, hbase and hive tokens cannot
-    // be obtained.
-    credentialManager.obtainCredentials(hadoopConf, creds)
-    val tokens = creds.getAllTokens
-    tokens.size() should be (1)
-    tokens.iterator().next().getService should be (new Text("test"))
-  }
-
-  test("verify getting credential renewal info") {
-    credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
-    val creds = new Credentials()
-
-    val testCredentialProvider = credentialManager.getServiceCredentialProvider("test").get
-      .asInstanceOf[TestCredentialProvider]
-    // Only TestTokenProvider can get the time of next token renewal
-    val nextRenewal = credentialManager.obtainCredentials(hadoopConf, creds)
-    nextRenewal should be (testCredentialProvider.timeOfNextTokenRenewal)
-  }
-
-  test("obtain tokens For HiveMetastore") {
-    val hadoopConf = new Configuration()
-    hadoopConf.set("hive.metastore.kerberos.principal", "bob")
-    // thrift picks up on port 0 and bails out, without trying to talk to endpoint
-    hadoopConf.set("hive.metastore.uris", "http://localhost:0")
-
-    val hiveCredentialProvider = new HiveCredentialProvider()
-    val credentials = new Credentials()
-    hiveCredentialProvider.obtainCredentials(hadoopConf, sparkConf, credentials)
-
-    credentials.getAllTokens.size() should be (0)
-  }
-
-  test("Obtain tokens For HBase") {
-    val hadoopConf = new Configuration()
-    hadoopConf.set("hbase.security.authentication", "kerberos")
-
-    val hbaseTokenProvider = new HBaseCredentialProvider()
-    val creds = new Credentials()
-    hbaseTokenProvider.obtainCredentials(hadoopConf, sparkConf, creds)
-
-    creds.getAllTokens.size should be (0)
-  }
-}
-
-class TestCredentialProvider extends ServiceCredentialProvider {
-  val tokenRenewalInterval = 86400 * 1000L
-  var timeOfNextTokenRenewal = 0L
-
-  override def serviceName: String = "test"
-
-  override def credentialsRequired(conf: Configuration): Boolean = true
-
-  override def obtainCredentials(
-      hadoopConf: Configuration,
-      sparkConf: SparkConf,
-      creds: Credentials): Option[Long] = {
-    if (creds == null) {
-      // Guard out other unit test failures.
-      return None
-    }
-
-    val emptyToken = new Token()
-    emptyToken.setService(new Text("test"))
-    creds.addToken(emptyToken.getService, emptyToken)
-
-    val currTime = System.currentTimeMillis()
-    timeOfNextTokenRenewal = (currTime - currTime % tokenRenewalInterval) + tokenRenewalInterval
-
-    Some(timeOfNextTokenRenewal)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProviderSuite.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProviderSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProviderSuite.scala
deleted file mode 100644
index 7b2da3f..0000000
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProviderSuite.scala
+++ /dev/null
@@ -1,71 +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.deploy.yarn.security
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.scalatest.{Matchers, PrivateMethodTester}
-
-import org.apache.spark.{SparkConf, SparkException, SparkFunSuite}
-
-class HDFSCredentialProviderSuite
-    extends SparkFunSuite
-    with PrivateMethodTester
-    with Matchers {
-  private val _getTokenRenewer = PrivateMethod[String]('getTokenRenewer)
-
-  private def getTokenRenewer(
-      hdfsCredentialProvider: HDFSCredentialProvider, conf: Configuration): String = {
-    hdfsCredentialProvider invokePrivate _getTokenRenewer(conf)
-  }
-
-  private var hdfsCredentialProvider: HDFSCredentialProvider = null
-
-  override def beforeAll() {
-    super.beforeAll()
-
-    if (hdfsCredentialProvider == null) {
-      hdfsCredentialProvider = new HDFSCredentialProvider()
-    }
-  }
-
-  override def afterAll() {
-    if (hdfsCredentialProvider != null) {
-      hdfsCredentialProvider = null
-    }
-
-    super.afterAll()
-  }
-
-  test("check token renewer") {
-    val hadoopConf = new Configuration()
-    hadoopConf.set("yarn.resourcemanager.address", "myrm:8033")
-    hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM")
-    val renewer = getTokenRenewer(hdfsCredentialProvider, hadoopConf)
-    renewer should be ("yarn/myrm:8032@SPARKTEST.COM")
-  }
-
-  test("check token renewer default") {
-    val hadoopConf = new Configuration()
-    val caught =
-      intercept[SparkException] {
-        getTokenRenewer(hdfsCredentialProvider, hadoopConf)
-      }
-    assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer")
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala b/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala
deleted file mode 100644
index da9e8e2..0000000
--- a/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala
+++ /dev/null
@@ -1,36 +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.launcher
-
-import java.util.{List => JList, Map => JMap}
-
-/**
- * Exposes AbstractCommandBuilder to the YARN tests, so that they can build classpaths the same
- * way other cluster managers do.
- */
-private[spark] class TestClasspathBuilder extends AbstractCommandBuilder {
-
-  childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, sys.props("spark.test.home"))
-
-  override def buildClassPath(extraCp: String): JList[String] = super.buildClassPath(extraCp)
-
-  /** Not used by the YARN tests. */
-  override def buildCommand(env: JMap[String, String]): JList[String] =
-    throw new UnsupportedOperationException()
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala b/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala
deleted file mode 100644
index 1fed256..0000000
--- a/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala
+++ /dev/null
@@ -1,70 +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.network.shuffle
-
-import java.io.File
-import java.util.concurrent.ConcurrentMap
-
-import org.apache.hadoop.yarn.api.records.ApplicationId
-import org.fusesource.leveldbjni.JniDBFactory
-import org.iq80.leveldb.{DB, Options}
-
-import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId
-import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo
-
-/**
- * just a cheat to get package-visible members in tests
- */
-object ShuffleTestAccessor {
-
-  def getBlockResolver(handler: ExternalShuffleBlockHandler): ExternalShuffleBlockResolver = {
-    handler.blockManager
-  }
-
-  def getExecutorInfo(
-      appId: ApplicationId,
-      execId: String,
-      resolver: ExternalShuffleBlockResolver
-  ): Option[ExecutorShuffleInfo] = {
-    val id = new AppExecId(appId.toString, execId)
-    Option(resolver.executors.get(id))
-  }
-
-  def registeredExecutorFile(resolver: ExternalShuffleBlockResolver): File = {
-    resolver.registeredExecutorFile
-  }
-
-  def shuffleServiceLevelDB(resolver: ExternalShuffleBlockResolver): DB = {
-    resolver.db
-  }
-
-  def reloadRegisteredExecutors(
-    file: File): ConcurrentMap[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo] = {
-    val options: Options = new Options
-    options.createIfMissing(true)
-    val factory = new JniDBFactory
-    val db = factory.open(file, options)
-    val result = ExternalShuffleBlockResolver.reloadRegisteredExecutors(db)
-    db.close()
-    result
-  }
-
-  def reloadRegisteredExecutors(
-      db: DB): ConcurrentMap[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo] = {
-    ExternalShuffleBlockResolver.reloadRegisteredExecutors(db)
-  }
-}


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


[04/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/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
deleted file mode 100644
index 8772e26..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
+++ /dev/null
@@ -1,224 +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.deploy.yarn
-
-import scala.collection.mutable.{ArrayBuffer, HashMap, Set}
-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._
-
-private[yarn] case class ContainerLocalityPreferences(nodes: Array[String], racks: Array[String])
-
-/**
- * This strategy is calculating the optimal locality preferences of YARN containers by considering
- * the node ratio of pending tasks, number of required cores/containers and and locality of current
- * existing and pending allocated containers. The target of this algorithm is to maximize the number
- * of tasks that would run locally.
- *
- * Consider a situation in which we have 20 tasks that require (host1, host2, host3)
- * and 10 tasks that require (host1, host2, host4), besides each container has 2 cores
- * and cpus per task is 1, so the required container number is 15,
- * and host ratio is (host1: 30, host2: 30, host3: 20, host4: 10).
- *
- * 1. If requested container number (18) is more than the required container number (15):
- *
- * requests for 5 containers with nodes: (host1, host2, host3, host4)
- * requests for 5 containers with nodes: (host1, host2, host3)
- * requests for 5 containers with nodes: (host1, host2)
- * requests for 3 containers with no locality preferences.
- *
- * The placement ratio is 3 : 3 : 2 : 1, and set the additional containers with no locality
- * preferences.
- *
- * 2. If requested container number (10) is less than or equal to the required container number
- * (15):
- *
- * requests for 4 containers with nodes: (host1, host2, host3, host4)
- * requests for 3 containers with nodes: (host1, host2, host3)
- * requests for 3 containers with nodes: (host1, host2)
- *
- * The placement ratio is 10 : 10 : 7 : 4, close to expected ratio (3 : 3 : 2 : 1)
- *
- * 3. If containers exist but none of them can match the requested localities,
- * follow the method of 1 and 2.
- *
- * 4. If containers exist and some of them can match the requested localities.
- * For example if we have 1 containers on each node (host1: 1, host2: 1: host3: 1, host4: 1),
- * and the expected containers on each node would be (host1: 5, host2: 5, host3: 4, host4: 2),
- * so the newly requested containers on each node would be updated to (host1: 4, host2: 4,
- * host3: 3, host4: 1), 12 containers by total.
- *
- *   4.1 If requested container number (18) is more than newly required containers (12). Follow
- *   method 1 with updated ratio 4 : 4 : 3 : 1.
- *
- *   4.2 If request container number (10) is more than newly required containers (12). Follow
- *   method 2 with updated ratio 4 : 4 : 3 : 1.
- *
- * 5. If containers exist and existing localities can fully cover the requested localities.
- * For example if we have 5 containers on each node (host1: 5, host2: 5, host3: 5, host4: 5),
- * which could cover the current requested localities. This algorithm will allocate all the
- * requested containers with no localities.
- */
-private[yarn] class LocalityPreferredContainerPlacementStrategy(
-    val sparkConf: SparkConf,
-    val yarnConf: Configuration,
-    val resource: Resource) {
-
-  /**
-   * Calculate each container's node locality and rack locality
-   * @param numContainer number of containers to calculate
-   * @param numLocalityAwareTasks number of locality required tasks
-   * @param hostToLocalTaskCount a map to store the preferred hostname and possible task
-   *                             numbers running on it, used as hints for container allocation
-   * @param allocatedHostToContainersMap host to allocated containers map, used to calculate the
-   *                                     expected locality preference by considering the existing
-   *                                     containers
-   * @param localityMatchedPendingAllocations A sequence of pending container request which
-   *                                          matches the localities of current required tasks.
-   * @return node localities and rack localities, each locality is an array of string,
-   *         the length of localities is the same as number of containers
-   */
-  def localityOfRequestedContainers(
-      numContainer: Int,
-      numLocalityAwareTasks: Int,
-      hostToLocalTaskCount: Map[String, Int],
-      allocatedHostToContainersMap: HashMap[String, Set[ContainerId]],
-      localityMatchedPendingAllocations: Seq[ContainerRequest]
-    ): Array[ContainerLocalityPreferences] = {
-    val updatedHostToContainerCount = expectedHostToContainerCount(
-      numLocalityAwareTasks, hostToLocalTaskCount, allocatedHostToContainersMap,
-        localityMatchedPendingAllocations)
-    val updatedLocalityAwareContainerNum = updatedHostToContainerCount.values.sum
-
-    // The number of containers to allocate, divided into two groups, one with preferred locality,
-    // and the other without locality preference.
-    val requiredLocalityFreeContainerNum =
-      math.max(0, numContainer - updatedLocalityAwareContainerNum)
-    val requiredLocalityAwareContainerNum = numContainer - requiredLocalityFreeContainerNum
-
-    val containerLocalityPreferences = ArrayBuffer[ContainerLocalityPreferences]()
-    if (requiredLocalityFreeContainerNum > 0) {
-      for (i <- 0 until requiredLocalityFreeContainerNum) {
-        containerLocalityPreferences += ContainerLocalityPreferences(
-          null.asInstanceOf[Array[String]], null.asInstanceOf[Array[String]])
-      }
-    }
-
-    if (requiredLocalityAwareContainerNum > 0) {
-      val largestRatio = updatedHostToContainerCount.values.max
-      // Round the ratio of preferred locality to the number of locality required container
-      // number, which is used for locality preferred host calculating.
-      var preferredLocalityRatio = updatedHostToContainerCount.mapValues { ratio =>
-        val adjustedRatio = ratio.toDouble * requiredLocalityAwareContainerNum / largestRatio
-        adjustedRatio.ceil.toInt
-      }
-
-      for (i <- 0 until requiredLocalityAwareContainerNum) {
-        // 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 =>
-          RackResolver.resolve(yarnConf, h).getNetworkLocation
-        }.toSet
-        containerLocalityPreferences += ContainerLocalityPreferences(hosts, racks.toArray)
-
-        // Minus 1 each time when the host is used. When the current ratio is 0,
-        // which means all the required ratio is satisfied, this host will not be allocated again.
-        preferredLocalityRatio = preferredLocalityRatio.mapValues(_ - 1)
-      }
-    }
-
-    containerLocalityPreferences.toArray
-  }
-
-  /**
-   * Calculate the number of executors need to satisfy the given number of pending tasks.
-   */
-  private def numExecutorsPending(numTasksPending: Int): Int = {
-    val coresPerExecutor = resource.getVirtualCores
-    (numTasksPending * sparkConf.get(CPUS_PER_TASK) + coresPerExecutor - 1) / coresPerExecutor
-  }
-
-  /**
-   * Calculate the expected host to number of containers by considering with allocated containers.
-   * @param localityAwareTasks number of locality aware tasks
-   * @param hostToLocalTaskCount a map to store the preferred hostname and possible task
-   *                             numbers running on it, used as hints for container allocation
-   * @param allocatedHostToContainersMap host to allocated containers map, used to calculate the
-   *                                     expected locality preference by considering the existing
-   *                                     containers
-   * @param localityMatchedPendingAllocations A sequence of pending container request which
-   *                                          matches the localities of current required tasks.
-   * @return a map with hostname as key and required number of containers on this host as value
-   */
-  private def expectedHostToContainerCount(
-      localityAwareTasks: Int,
-      hostToLocalTaskCount: Map[String, Int],
-      allocatedHostToContainersMap: HashMap[String, Set[ContainerId]],
-      localityMatchedPendingAllocations: Seq[ContainerRequest]
-    ): Map[String, Int] = {
-    val totalLocalTaskNum = hostToLocalTaskCount.values.sum
-    val pendingHostToContainersMap = pendingHostToContainerCount(localityMatchedPendingAllocations)
-
-    hostToLocalTaskCount.map { case (host, count) =>
-      val expectedCount =
-        count.toDouble * numExecutorsPending(localityAwareTasks) / totalLocalTaskNum
-      // Take the locality of pending containers into consideration
-      val existedCount = allocatedHostToContainersMap.get(host).map(_.size).getOrElse(0) +
-        pendingHostToContainersMap.getOrElse(host, 0.0)
-
-      // If existing container can not fully satisfy the expected number of container,
-      // the required container number is expected count minus existed count. Otherwise the
-      // required container number is 0.
-      (host, math.max(0, (expectedCount - existedCount).ceil.toInt))
-    }
-  }
-
-  /**
-   * According to the locality ratio and number of container requests, calculate the host to
-   * possible number of containers for pending allocated containers.
-   *
-   * If current locality ratio of hosts is: Host1 : Host2 : Host3 = 20 : 20 : 10,
-   * and pending container requests is 3, so the possible number of containers on
-   * Host1 : Host2 : Host3 will be 1.2 : 1.2 : 0.6.
-   * @param localityMatchedPendingAllocations A sequence of pending container request which
-   *                                          matches the localities of current required tasks.
-   * @return a Map with hostname as key and possible number of containers on this host as value
-   */
-  private def pendingHostToContainerCount(
-      localityMatchedPendingAllocations: Seq[ContainerRequest]): Map[String, Double] = {
-    val pendingHostToContainerCount = new HashMap[String, Int]()
-    localityMatchedPendingAllocations.foreach { cr =>
-      cr.getNodes.asScala.foreach { n =>
-        val count = pendingHostToContainerCount.getOrElse(n, 0) + 1
-        pendingHostToContainerCount(n) = count
-      }
-    }
-
-    val possibleTotalContainerNum = pendingHostToContainerCount.values.sum
-    val localityMatchedPendingNum = localityMatchedPendingAllocations.size.toDouble
-    pendingHostToContainerCount.mapValues(_ * localityMatchedPendingNum / possibleTotalContainerNum)
-      .toMap
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/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
deleted file mode 100644
index 0b66d1c..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
+++ /dev/null
@@ -1,727 +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.deploy.yarn
-
-import java.util.Collections
-import java.util.concurrent._
-import java.util.regex.Pattern
-
-import scala.collection.mutable
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue}
-import scala.collection.JavaConverters._
-import scala.util.control.NonFatal
-
-import org.apache.hadoop.yarn.api.records._
-import org.apache.hadoop.yarn.client.api.AMRMClient
-import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.yarn.util.RackResolver
-import org.apache.log4j.{Level, Logger}
-
-import org.apache.spark.{SecurityManager, SparkConf, SparkException}
-import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.internal.Logging
-import org.apache.spark.internal.config._
-import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef}
-import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason}
-import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor
-import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RetrieveLastAllocatedExecutorId
-import org.apache.spark.util.{Clock, SystemClock, ThreadUtils}
-
-/**
- * YarnAllocator is charged with requesting containers from the YARN ResourceManager and deciding
- * what to do with containers when YARN fulfills these requests.
- *
- * This class makes use of YARN's AMRMClient APIs. We interact with the AMRMClient in three ways:
- * * Making our resource needs known, which updates local bookkeeping about containers requested.
- * * Calling "allocate", which syncs our local container requests with the RM, and returns any
- *   containers that YARN has granted to us.  This also functions as a heartbeat.
- * * Processing the containers granted to us to possibly launch executors inside of them.
- *
- * The public methods of this class are thread-safe.  All methods that mutate state are
- * synchronized.
- */
-private[yarn] class YarnAllocator(
-    driverUrl: String,
-    driverRef: RpcEndpointRef,
-    conf: YarnConfiguration,
-    sparkConf: SparkConf,
-    amClient: AMRMClient[ContainerRequest],
-    appAttemptId: ApplicationAttemptId,
-    securityMgr: SecurityManager,
-    localResources: Map[String, LocalResource])
-  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]
-
-  // Containers that we no longer care about. We've either already told the RM to release them or
-  // will on the next heartbeat. Containers get removed from this map after the RM tells us they've
-  // completed.
-  private val releasedContainers = Collections.newSetFromMap[ContainerId](
-    new ConcurrentHashMap[ContainerId, java.lang.Boolean])
-
-  @volatile private var numExecutorsRunning = 0
-
-  /**
-   * Used to generate a unique ID per executor
-   *
-   * Init `executorIdCounter`. when AM restart, `executorIdCounter` will reset to 0. Then
-   * the id of new executor will start from 1, this will conflict with the executor has
-   * already created before. So, we should initialize the `executorIdCounter` by getting
-   * the max executorId from driver.
-   *
-   * And this situation of executorId conflict is just in yarn client mode, so this is an issue
-   * in yarn client mode. For more details, can check in jira.
-   *
-   * @see SPARK-12864
-   */
-  private var executorIdCounter: Int =
-    driverRef.askWithRetry[Int](RetrieveLastAllocatedExecutorId)
-
-  // Queue to store the timestamp of failed executors
-  private val failedExecutorsTimeStamps = new Queue[Long]()
-
-  private var clock: Clock = new SystemClock
-
-  private val executorFailuresValidityInterval =
-    sparkConf.get(EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS).getOrElse(-1L)
-
-  @volatile private var targetNumExecutors =
-    YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sparkConf)
-
-  // Executor loss reason requests that are pending - maps from executor ID for inquiry to a
-  // list of requesters that should be responded to once we find out why the given executor
-  // was lost.
-  private val pendingLossReasonRequests = new HashMap[String, mutable.Buffer[RpcCallContext]]
-
-  // Maintain loss reasons for already released executors, it will be added when executor loss
-  // reason is got from AM-RM call, and be removed after querying this loss reason.
-  private val releasedExecutorLossReasons = new HashMap[String, ExecutorLossReason]
-
-  // Keep track of which container is running which executor to remove the executors later
-  // Visible for testing.
-  private[yarn] val executorIdToContainer = new HashMap[String, Container]
-
-  private var numUnexpectedContainerRelease = 0L
-  private val containerIdToExecutorId = new HashMap[ContainerId, String]
-
-  // Executor memory in MB.
-  protected val executorMemory = sparkConf.get(EXECUTOR_MEMORY).toInt
-  // Additional memory overhead.
-  protected val memoryOverhead: Int = sparkConf.get(EXECUTOR_MEMORY_OVERHEAD).getOrElse(
-    math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toInt, MEMORY_OVERHEAD_MIN)).toInt
-  // Number of cores per executor.
-  protected val executorCores = sparkConf.get(EXECUTOR_CORES)
-  // Resource capability requested for each executors
-  private[yarn] val resource = Resource.newInstance(executorMemory + memoryOverhead, executorCores)
-
-  private val launcherPool = ThreadUtils.newDaemonCachedThreadPool(
-    "ContainerLauncher", sparkConf.get(CONTAINER_LAUNCH_MAX_THREADS))
-
-  // For testing
-  private val launchContainers = sparkConf.getBoolean("spark.yarn.launchContainers", true)
-
-  private val labelExpression = sparkConf.get(EXECUTOR_NODE_LABEL_EXPRESSION)
-
-  // ContainerRequest constructor that can take a node label expression. We grab it through
-  // reflection because it's only available in later versions of YARN.
-  private val nodeLabelConstructor = labelExpression.flatMap { expr =>
-    try {
-      Some(classOf[ContainerRequest].getConstructor(classOf[Resource],
-        classOf[Array[String]], classOf[Array[String]], classOf[Priority], classOf[Boolean],
-        classOf[String]))
-    } catch {
-      case e: NoSuchMethodException =>
-        logWarning(s"Node label expression $expr will be ignored because YARN version on" +
-          " classpath does not support it.")
-        None
-    }
-  }
-
-  // A map to store preferred hostname and possible task numbers running on it.
-  private var hostToLocalTaskCounts: Map[String, Int] = Map.empty
-
-  // Number of tasks that have locality preferences in active stages
-  private var numLocalityAwareTasks: Int = 0
-
-  // A container placement strategy based on pending tasks' locality preference
-  private[yarn] val containerPlacementStrategy =
-    new LocalityPreferredContainerPlacementStrategy(sparkConf, conf, resource)
-
-  /**
-   * Use a different clock for YarnAllocator. This is mainly used for testing.
-   */
-  def setClock(newClock: Clock): Unit = {
-    clock = newClock
-  }
-
-  def getNumExecutorsRunning: Int = numExecutorsRunning
-
-  def getNumExecutorsFailed: Int = synchronized {
-    val endTime = clock.getTimeMillis()
-
-    while (executorFailuresValidityInterval > 0
-      && failedExecutorsTimeStamps.nonEmpty
-      && failedExecutorsTimeStamps.head < endTime - executorFailuresValidityInterval) {
-      failedExecutorsTimeStamps.dequeue()
-    }
-
-    failedExecutorsTimeStamps.size
-  }
-
-  /**
-   * A sequence of pending container requests that have not yet been fulfilled.
-   */
-  def getPendingAllocate: Seq[ContainerRequest] = getPendingAtLocation(ANY_HOST)
-
-  /**
-   * A sequence of pending container requests at the given location that have not yet been
-   * fulfilled.
-   */
-  private def getPendingAtLocation(location: String): Seq[ContainerRequest] = {
-    amClient.getMatchingRequests(RM_REQUEST_PRIORITY, location, resource).asScala
-      .flatMap(_.asScala)
-      .toSeq
-  }
-
-  /**
-   * Request as many executors from the ResourceManager as needed to reach the desired total. If
-   * the requested total is smaller than the current number of running executors, no executors will
-   * be killed.
-   * @param requestedTotal total number of containers requested
-   * @param localityAwareTasks number of locality aware tasks to be used as container placement hint
-   * @param hostToLocalTaskCount a map of preferred hostname to possible task counts to be used as
-   *                             container placement hint.
-   * @return Whether the new requested total is different than the old value.
-   */
-  def requestTotalExecutorsWithPreferredLocalities(
-      requestedTotal: Int,
-      localityAwareTasks: Int,
-      hostToLocalTaskCount: Map[String, Int]): Boolean = synchronized {
-    this.numLocalityAwareTasks = localityAwareTasks
-    this.hostToLocalTaskCounts = hostToLocalTaskCount
-
-    if (requestedTotal != targetNumExecutors) {
-      logInfo(s"Driver requested a total number of $requestedTotal executor(s).")
-      targetNumExecutors = requestedTotal
-      true
-    } else {
-      false
-    }
-  }
-
-  /**
-   * Request that the ResourceManager release the container running the specified executor.
-   */
-  def killExecutor(executorId: String): Unit = synchronized {
-    if (executorIdToContainer.contains(executorId)) {
-      val container = executorIdToContainer.get(executorId).get
-      internalReleaseContainer(container)
-      numExecutorsRunning -= 1
-    } else {
-      logWarning(s"Attempted to kill unknown executor $executorId!")
-    }
-  }
-
-  /**
-   * Request resources such that, if YARN gives us all we ask for, we'll have a number of containers
-   * equal to maxExecutors.
-   *
-   * Deal with any containers YARN has granted to us by possibly launching executors in them.
-   *
-   * This must be synchronized because variables read in this method are mutated by other methods.
-   */
-  def allocateResources(): Unit = synchronized {
-    updateResourceRequests()
-
-    val progressIndicator = 0.1f
-    // Poll the ResourceManager. This doubles as a heartbeat if there are no pending container
-    // requests.
-    val allocateResponse = amClient.allocate(progressIndicator)
-
-    val allocatedContainers = allocateResponse.getAllocatedContainers()
-
-    if (allocatedContainers.size > 0) {
-      logDebug("Allocated containers: %d. Current executor count: %d. Cluster resources: %s."
-        .format(
-          allocatedContainers.size,
-          numExecutorsRunning,
-          allocateResponse.getAvailableResources))
-
-      handleAllocatedContainers(allocatedContainers.asScala)
-    }
-
-    val completedContainers = allocateResponse.getCompletedContainersStatuses()
-    if (completedContainers.size > 0) {
-      logDebug("Completed %d containers".format(completedContainers.size))
-      processCompletedContainers(completedContainers.asScala)
-      logDebug("Finished processing %d completed containers. Current running executor count: %d."
-        .format(completedContainers.size, numExecutorsRunning))
-    }
-  }
-
-  /**
-   * Update the set of container requests that we will sync with the RM based on the number of
-   * executors we have currently running and our target number of executors.
-   *
-   * Visible for testing.
-   */
-  def updateResourceRequests(): Unit = {
-    val pendingAllocate = getPendingAllocate
-    val numPendingAllocate = pendingAllocate.size
-    val missing = targetNumExecutors - numPendingAllocate - numExecutorsRunning
-
-    if (missing > 0) {
-      logInfo(s"Will request $missing executor container(s), each with " +
-        s"${resource.getVirtualCores} core(s) and " +
-        s"${resource.getMemory} MB memory (including $memoryOverhead MB of overhead)")
-
-      // Split the pending container request into three groups: locality matched list, locality
-      // unmatched list and non-locality list. Take the locality matched container request into
-      // consideration of container placement, treat as allocated containers.
-      // For locality unmatched and locality free container requests, cancel these container
-      // requests, since required locality preference has been changed, recalculating using
-      // container placement strategy.
-      val (localRequests, staleRequests, anyHostRequests) = splitPendingAllocationsByLocality(
-        hostToLocalTaskCounts, pendingAllocate)
-
-      // cancel "stale" requests for locations that are no longer needed
-      staleRequests.foreach { stale =>
-        amClient.removeContainerRequest(stale)
-      }
-      val cancelledContainers = staleRequests.size
-      if (cancelledContainers > 0) {
-        logInfo(s"Canceled $cancelledContainers container request(s) (locality no longer needed)")
-      }
-
-      // consider the number of new containers and cancelled stale containers available
-      val availableContainers = missing + cancelledContainers
-
-      // to maximize locality, include requests with no locality preference that can be cancelled
-      val potentialContainers = availableContainers + anyHostRequests.size
-
-      val containerLocalityPreferences = containerPlacementStrategy.localityOfRequestedContainers(
-        potentialContainers, numLocalityAwareTasks, hostToLocalTaskCounts,
-          allocatedHostToContainersMap, localRequests)
-
-      val newLocalityRequests = new mutable.ArrayBuffer[ContainerRequest]
-      containerLocalityPreferences.foreach {
-        case ContainerLocalityPreferences(nodes, racks) if nodes != null =>
-          newLocalityRequests += createContainerRequest(resource, nodes, racks)
-        case _ =>
-      }
-
-      if (availableContainers >= newLocalityRequests.size) {
-        // more containers are available than needed for locality, fill in requests for any host
-        for (i <- 0 until (availableContainers - newLocalityRequests.size)) {
-          newLocalityRequests += createContainerRequest(resource, null, null)
-        }
-      } else {
-        val numToCancel = newLocalityRequests.size - availableContainers
-        // cancel some requests without locality preferences to schedule more local containers
-        anyHostRequests.slice(0, numToCancel).foreach { nonLocal =>
-          amClient.removeContainerRequest(nonLocal)
-        }
-        if (numToCancel > 0) {
-          logInfo(s"Canceled $numToCancel unlocalized container requests to resubmit with locality")
-        }
-      }
-
-      newLocalityRequests.foreach { request =>
-        amClient.addContainerRequest(request)
-      }
-
-      if (log.isInfoEnabled()) {
-        val (localized, anyHost) = newLocalityRequests.partition(_.getNodes() != null)
-        if (anyHost.nonEmpty) {
-          logInfo(s"Submitted ${anyHost.size} unlocalized container requests.")
-        }
-        localized.foreach { request =>
-          logInfo(s"Submitted container request for host ${hostStr(request)}.")
-        }
-      }
-    } else if (numPendingAllocate > 0 && missing < 0) {
-      val numToCancel = math.min(numPendingAllocate, -missing)
-      logInfo(s"Canceling requests for $numToCancel executor container(s) to have a new desired " +
-        s"total $targetNumExecutors executors.")
-
-      val matchingRequests = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, ANY_HOST, resource)
-      if (!matchingRequests.isEmpty) {
-        matchingRequests.iterator().next().asScala
-          .take(numToCancel).foreach(amClient.removeContainerRequest)
-      } else {
-        logWarning("Expected to find pending requests, but found none.")
-      }
-    }
-  }
-
-  private def hostStr(request: ContainerRequest): String = {
-    Option(request.getNodes) match {
-      case Some(nodes) => nodes.asScala.mkString(",")
-      case None => "Any"
-    }
-  }
-
-  /**
-   * Creates a container request, handling the reflection required to use YARN features that were
-   * added in recent versions.
-   */
-  private def createContainerRequest(
-      resource: Resource,
-      nodes: Array[String],
-      racks: Array[String]): ContainerRequest = {
-    nodeLabelConstructor.map { constructor =>
-      constructor.newInstance(resource, nodes, racks, RM_REQUEST_PRIORITY, true: java.lang.Boolean,
-        labelExpression.orNull)
-    }.getOrElse(new ContainerRequest(resource, nodes, racks, RM_REQUEST_PRIORITY))
-  }
-
-  /**
-   * Handle containers granted by the RM by launching executors on them.
-   *
-   * Due to the way the YARN allocation protocol works, certain healthy race conditions can result
-   * in YARN granting containers that we no longer need. In this case, we release them.
-   *
-   * Visible for testing.
-   */
-  def handleAllocatedContainers(allocatedContainers: Seq[Container]): Unit = {
-    val containersToUse = new ArrayBuffer[Container](allocatedContainers.size)
-
-    // Match incoming requests by host
-    val remainingAfterHostMatches = new ArrayBuffer[Container]
-    for (allocatedContainer <- allocatedContainers) {
-      matchContainerToRequest(allocatedContainer, allocatedContainer.getNodeId.getHost,
-        containersToUse, remainingAfterHostMatches)
-    }
-
-    // Match remaining by rack
-    val remainingAfterRackMatches = new ArrayBuffer[Container]
-    for (allocatedContainer <- remainingAfterHostMatches) {
-      val rack = RackResolver.resolve(conf, allocatedContainer.getNodeId.getHost).getNetworkLocation
-      matchContainerToRequest(allocatedContainer, rack, containersToUse,
-        remainingAfterRackMatches)
-    }
-
-    // Assign remaining that are neither node-local nor rack-local
-    val remainingAfterOffRackMatches = new ArrayBuffer[Container]
-    for (allocatedContainer <- remainingAfterRackMatches) {
-      matchContainerToRequest(allocatedContainer, ANY_HOST, containersToUse,
-        remainingAfterOffRackMatches)
-    }
-
-    if (!remainingAfterOffRackMatches.isEmpty) {
-      logDebug(s"Releasing ${remainingAfterOffRackMatches.size} unneeded containers that were " +
-        s"allocated to us")
-      for (container <- remainingAfterOffRackMatches) {
-        internalReleaseContainer(container)
-      }
-    }
-
-    runAllocatedContainers(containersToUse)
-
-    logInfo("Received %d containers from YARN, launching executors on %d of them."
-      .format(allocatedContainers.size, containersToUse.size))
-  }
-
-  /**
-   * Looks for requests for the given location that match the given container allocation. If it
-   * finds one, removes the request so that it won't be submitted again. Places the container into
-   * containersToUse or remaining.
-   *
-   * @param allocatedContainer container that was given to us by YARN
-   * @param location resource name, either a node, rack, or *
-   * @param containersToUse list of containers that will be used
-   * @param remaining list of containers that will not be used
-   */
-  private def matchContainerToRequest(
-      allocatedContainer: Container,
-      location: String,
-      containersToUse: ArrayBuffer[Container],
-      remaining: ArrayBuffer[Container]): Unit = {
-    // SPARK-6050: certain Yarn configurations return a virtual core count that doesn't match the
-    // request; for example, capacity scheduler + DefaultResourceCalculator. So match on requested
-    // memory, but use the asked vcore count for matching, effectively disabling matching on vcore
-    // count.
-    val matchingResource = Resource.newInstance(allocatedContainer.getResource.getMemory,
-          resource.getVirtualCores)
-    val matchingRequests = amClient.getMatchingRequests(allocatedContainer.getPriority, location,
-      matchingResource)
-
-    // Match the allocation to a request
-    if (!matchingRequests.isEmpty) {
-      val containerRequest = matchingRequests.get(0).iterator.next
-      amClient.removeContainerRequest(containerRequest)
-      containersToUse += allocatedContainer
-    } else {
-      remaining += allocatedContainer
-    }
-  }
-
-  /**
-   * Launches executors in the allocated containers.
-   */
-  private def runAllocatedContainers(containersToUse: ArrayBuffer[Container]): Unit = {
-    for (container <- containersToUse) {
-      executorIdCounter += 1
-      val executorHostname = container.getNodeId.getHost
-      val containerId = container.getId
-      val executorId = executorIdCounter.toString
-      assert(container.getResource.getMemory >= resource.getMemory)
-      logInfo(s"Launching container $containerId on host $executorHostname")
-
-      def updateInternalState(): Unit = synchronized {
-        numExecutorsRunning += 1
-        executorIdToContainer(executorId) = container
-        containerIdToExecutorId(container.getId) = executorId
-
-        val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname,
-          new HashSet[ContainerId])
-        containerSet += containerId
-        allocatedContainerToHostMap.put(containerId, executorHostname)
-      }
-
-      if (numExecutorsRunning < targetNumExecutors) {
-        if (launchContainers) {
-          launcherPool.execute(new Runnable {
-            override def run(): Unit = {
-              try {
-                new ExecutorRunnable(
-                  Some(container),
-                  conf,
-                  sparkConf,
-                  driverUrl,
-                  executorId,
-                  executorHostname,
-                  executorMemory,
-                  executorCores,
-                  appAttemptId.getApplicationId.toString,
-                  securityMgr,
-                  localResources
-                ).run()
-                updateInternalState()
-              } catch {
-                case NonFatal(e) =>
-                  logError(s"Failed to launch executor $executorId on container $containerId", e)
-                  // Assigned container should be released immediately to avoid unnecessary resource
-                  // occupation.
-                  amClient.releaseAssignedContainer(containerId)
-              }
-            }
-          })
-        } else {
-          // For test only
-          updateInternalState()
-        }
-      } else {
-        logInfo(("Skip launching executorRunnable as runnning Excecutors count: %d " +
-          "reached target Executors count: %d.").format(numExecutorsRunning, targetNumExecutors))
-      }
-    }
-  }
-
-  // Visible for testing.
-  private[yarn] def processCompletedContainers(completedContainers: Seq[ContainerStatus]): Unit = {
-    for (completedContainer <- completedContainers) {
-      val containerId = completedContainer.getContainerId
-      val alreadyReleased = releasedContainers.remove(containerId)
-      val hostOpt = allocatedContainerToHostMap.get(containerId)
-      val onHostStr = hostOpt.map(host => s" on host: $host").getOrElse("")
-      val exitReason = if (!alreadyReleased) {
-        // Decrement the number of executors running. The next iteration of
-        // the ApplicationMaster's reporting thread will take care of allocating.
-        numExecutorsRunning -= 1
-        logInfo("Completed container %s%s (state: %s, exit status: %s)".format(
-          containerId,
-          onHostStr,
-          completedContainer.getState,
-          completedContainer.getExitStatus))
-        // Hadoop 2.2.X added a ContainerExitStatus we should switch to use
-        // there are some exit status' we shouldn't necessarily count against us, but for
-        // now I think its ok as none of the containers are expected to exit.
-        val exitStatus = completedContainer.getExitStatus
-        val (exitCausedByApp, containerExitReason) = exitStatus match {
-          case ContainerExitStatus.SUCCESS =>
-            (false, s"Executor for container $containerId exited because of a YARN event (e.g., " +
-              "pre-emption) and not because of an error in the running job.")
-          case ContainerExitStatus.PREEMPTED =>
-            // Preemption is not the fault of the running tasks, since YARN preempts containers
-            // merely to do resource sharing, and tasks that fail due to preempted executors could
-            // just as easily finish on any other executor. See SPARK-8167.
-            (false, s"Container ${containerId}${onHostStr} was preempted.")
-          // Should probably still count memory exceeded exit codes towards task failures
-          case VMEM_EXCEEDED_EXIT_CODE =>
-            (true, memLimitExceededLogMessage(
-              completedContainer.getDiagnostics,
-              VMEM_EXCEEDED_PATTERN))
-          case PMEM_EXCEEDED_EXIT_CODE =>
-            (true, memLimitExceededLogMessage(
-              completedContainer.getDiagnostics,
-              PMEM_EXCEEDED_PATTERN))
-          case _ =>
-            // Enqueue the timestamp of failed executor
-            failedExecutorsTimeStamps.enqueue(clock.getTimeMillis())
-            (true, "Container marked as failed: " + containerId + onHostStr +
-              ". Exit status: " + completedContainer.getExitStatus +
-              ". Diagnostics: " + completedContainer.getDiagnostics)
-
-        }
-        if (exitCausedByApp) {
-          logWarning(containerExitReason)
-        } else {
-          logInfo(containerExitReason)
-        }
-        ExecutorExited(exitStatus, exitCausedByApp, containerExitReason)
-      } else {
-        // If we have already released this container, then it must mean
-        // that the driver has explicitly requested it to be killed
-        ExecutorExited(completedContainer.getExitStatus, exitCausedByApp = false,
-          s"Container $containerId exited from explicit termination request.")
-      }
-
-      for {
-        host <- hostOpt
-        containerSet <- allocatedHostToContainersMap.get(host)
-      } {
-        containerSet.remove(containerId)
-        if (containerSet.isEmpty) {
-          allocatedHostToContainersMap.remove(host)
-        } else {
-          allocatedHostToContainersMap.update(host, containerSet)
-        }
-
-        allocatedContainerToHostMap.remove(containerId)
-      }
-
-      containerIdToExecutorId.remove(containerId).foreach { eid =>
-        executorIdToContainer.remove(eid)
-        pendingLossReasonRequests.remove(eid) match {
-          case Some(pendingRequests) =>
-            // Notify application of executor loss reasons so it can decide whether it should abort
-            pendingRequests.foreach(_.reply(exitReason))
-
-          case None =>
-            // We cannot find executor for pending reasons. This is because completed container
-            // is processed before querying pending result. We should store it for later query.
-            // This is usually happened when explicitly killing a container, the result will be
-            // returned in one AM-RM communication. So query RPC will be later than this completed
-            // container process.
-            releasedExecutorLossReasons.put(eid, exitReason)
-        }
-        if (!alreadyReleased) {
-          // The executor could have gone away (like no route to host, node failure, etc)
-          // Notify backend about the failure of the executor
-          numUnexpectedContainerRelease += 1
-          driverRef.send(RemoveExecutor(eid, exitReason))
-        }
-      }
-    }
-  }
-
-  /**
-   * Register that some RpcCallContext has asked the AM why the executor was lost. Note that
-   * we can only find the loss reason to send back in the next call to allocateResources().
-   */
-  private[yarn] def enqueueGetLossReasonRequest(
-      eid: String,
-      context: RpcCallContext): Unit = synchronized {
-    if (executorIdToContainer.contains(eid)) {
-      pendingLossReasonRequests
-        .getOrElseUpdate(eid, new ArrayBuffer[RpcCallContext]) += context
-    } else if (releasedExecutorLossReasons.contains(eid)) {
-      // Executor is already released explicitly before getting the loss reason, so directly send
-      // the pre-stored lost reason
-      context.reply(releasedExecutorLossReasons.remove(eid).get)
-    } else {
-      logWarning(s"Tried to get the loss reason for non-existent executor $eid")
-      context.sendFailure(
-        new SparkException(s"Fail to find loss reason for non-existent executor $eid"))
-    }
-  }
-
-  private def internalReleaseContainer(container: Container): Unit = {
-    releasedContainers.add(container.getId())
-    amClient.releaseAssignedContainer(container.getId())
-  }
-
-  private[yarn] def getNumUnexpectedContainerRelease = numUnexpectedContainerRelease
-
-  private[yarn] def getNumPendingLossReasonRequests: Int = synchronized {
-    pendingLossReasonRequests.size
-  }
-
-  /**
-   * Split the pending container requests into 3 groups based on current localities of pending
-   * tasks.
-   * @param hostToLocalTaskCount a map of preferred hostname to possible task counts to be used as
-   *                             container placement hint.
-   * @param pendingAllocations A sequence of pending allocation container request.
-   * @return A tuple of 3 sequences, first is a sequence of locality matched container
-   *         requests, second is a sequence of locality unmatched container requests, and third is a
-   *         sequence of locality free container requests.
-   */
-  private def splitPendingAllocationsByLocality(
-      hostToLocalTaskCount: Map[String, Int],
-      pendingAllocations: Seq[ContainerRequest]
-    ): (Seq[ContainerRequest], Seq[ContainerRequest], Seq[ContainerRequest]) = {
-    val localityMatched = ArrayBuffer[ContainerRequest]()
-    val localityUnMatched = ArrayBuffer[ContainerRequest]()
-    val localityFree = ArrayBuffer[ContainerRequest]()
-
-    val preferredHosts = hostToLocalTaskCount.keySet
-    pendingAllocations.foreach { cr =>
-      val nodes = cr.getNodes
-      if (nodes == null) {
-        localityFree += cr
-      } else if (nodes.asScala.toSet.intersect(preferredHosts).nonEmpty) {
-        localityMatched += cr
-      } else {
-        localityUnMatched += cr
-      }
-    }
-
-    (localityMatched.toSeq, localityUnMatched.toSeq, localityFree.toSeq)
-  }
-
-}
-
-private object YarnAllocator {
-  val MEM_REGEX = "[0-9.]+ [KMG]B"
-  val PMEM_EXCEEDED_PATTERN =
-    Pattern.compile(s"$MEM_REGEX of $MEM_REGEX physical memory used")
-  val VMEM_EXCEEDED_PATTERN =
-    Pattern.compile(s"$MEM_REGEX of $MEM_REGEX virtual memory used")
-  val VMEM_EXCEEDED_EXIT_CODE = -103
-  val PMEM_EXCEEDED_EXIT_CODE = -104
-
-  def memLimitExceededLogMessage(diagnostics: String, pattern: Pattern): String = {
-    val matcher = pattern.matcher(diagnostics)
-    val diag = if (matcher.find()) " " + matcher.group() + "." else ""
-    ("Container killed by YARN for exceeding memory limits." + diag
-      + " Consider boosting spark.yarn.executor.memoryOverhead.")
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/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
deleted file mode 100644
index 53df11e..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
+++ /dev/null
@@ -1,135 +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.deploy.yarn
-
-import java.util.{List => JList}
-
-import scala.collection.JavaConverters._
-import scala.util.Try
-
-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
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.yarn.webapp.util.WebAppUtils
-
-import org.apache.spark.{SecurityManager, SparkConf}
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.internal.Logging
-import org.apache.spark.rpc.RpcEndpointRef
-import org.apache.spark.util.Utils
-
-/**
- * Handles registering and unregistering the application with the YARN ResourceManager.
- */
-private[spark] class YarnRMClient extends Logging {
-
-  private var amClient: AMRMClient[ContainerRequest] = _
-  private var uiHistoryAddress: String = _
-  private var registered: Boolean = false
-
-  /**
-   * Registers the application master with the RM.
-   *
-   * @param conf The Yarn configuration.
-   * @param sparkConf The Spark configuration.
-   * @param uiAddress Address of the SparkUI.
-   * @param uiHistoryAddress Address of the application on the History Server.
-   * @param securityMgr The security manager.
-   * @param localResources Map with information about files distributed via YARN's cache.
-   */
-  def register(
-      driverUrl: String,
-      driverRef: RpcEndpointRef,
-      conf: YarnConfiguration,
-      sparkConf: SparkConf,
-      uiAddress: String,
-      uiHistoryAddress: String,
-      securityMgr: SecurityManager,
-      localResources: Map[String, LocalResource]
-    ): YarnAllocator = {
-    amClient = AMRMClient.createAMRMClient()
-    amClient.init(conf)
-    amClient.start()
-    this.uiHistoryAddress = uiHistoryAddress
-
-    logInfo("Registering the ApplicationMaster")
-    synchronized {
-      amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress)
-      registered = true
-    }
-    new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, getAttemptId(), securityMgr,
-      localResources)
-  }
-
-  /**
-   * Unregister the AM. Guaranteed to only be called once.
-   *
-   * @param status The final status of the AM.
-   * @param diagnostics Diagnostics message to include in the final status.
-   */
-  def unregister(status: FinalApplicationStatus, diagnostics: String = ""): Unit = synchronized {
-    if (registered) {
-      amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress)
-    }
-  }
-
-  /** Returns the attempt ID. */
-  def getAttemptId(): ApplicationAttemptId = {
-    YarnSparkHadoopUtil.get.getContainerId.getApplicationAttemptId()
-  }
-
-  /** Returns the configuration for the AmIpFilter to add to the Spark UI. */
-  def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] = {
-    // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2,
-    // so not all stable releases have it.
-    val prefix = Try(classOf[WebAppUtils].getMethod("getHttpSchemePrefix", classOf[Configuration])
-      .invoke(null, conf).asInstanceOf[String]).getOrElse("http://")
-
-    // If running a new enough Yarn, use the HA-aware API for retrieving the RM addresses.
-    try {
-      val method = classOf[WebAppUtils].getMethod("getProxyHostsAndPortsForAmFilter",
-        classOf[Configuration])
-      val proxies = method.invoke(null, conf).asInstanceOf[JList[String]]
-      val hosts = proxies.asScala.map { proxy => proxy.split(":")(0) }
-      val uriBases = proxies.asScala.map { proxy => prefix + proxy + proxyBase }
-      Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(","))
-    } catch {
-      case e: NoSuchMethodException =>
-        val proxy = WebAppUtils.getProxyHostAndPort(conf)
-        val parts = proxy.split(":")
-        val uriBase = prefix + proxy + proxyBase
-        Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase)
-    }
-  }
-
-  /** Returns the maximum number of attempts to register the AM. */
-  def getMaxRegAttempts(sparkConf: SparkConf, yarnConf: YarnConfiguration): Int = {
-    val sparkMaxAttempts = sparkConf.get(MAX_APP_ATTEMPTS).map(_.toInt)
-    val yarnMaxAttempts = yarnConf.getInt(
-      YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)
-    val retval: Int = sparkMaxAttempts match {
-      case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts
-      case None => yarnMaxAttempts
-    }
-
-    retval
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
deleted file mode 100644
index cc53b1b..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
+++ /dev/null
@@ -1,317 +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.deploy.yarn
-
-import java.io.File
-import java.nio.charset.StandardCharsets.UTF_8
-import java.util.regex.Matcher
-import java.util.regex.Pattern
-
-import scala.collection.mutable.{HashMap, ListBuffer}
-import scala.util.Try
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.io.Text
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.security.Credentials
-import org.apache.hadoop.security.UserGroupInformation
-import org.apache.hadoop.yarn.api.ApplicationConstants
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-import org.apache.hadoop.yarn.api.records.{ApplicationAccessType, ContainerId, Priority}
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.yarn.util.ConverterUtils
-
-import org.apache.spark.{SecurityManager, SparkConf, SparkException}
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.deploy.yarn.security.{ConfigurableCredentialManager, CredentialUpdater}
-import org.apache.spark.internal.config._
-import org.apache.spark.launcher.YarnCommandBuilderUtils
-import org.apache.spark.util.Utils
-
-/**
- * Contains util methods to interact with Hadoop from spark.
- */
-class YarnSparkHadoopUtil extends SparkHadoopUtil {
-
-  private var credentialUpdater: CredentialUpdater = _
-
-  override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) {
-    dest.addCredentials(source.getCredentials())
-  }
-
-  // Note that all params which start with SPARK are propagated all the way through, so if in yarn
-  // mode, this MUST be set to true.
-  override def isYarnMode(): Boolean = { true }
-
-  // Return an appropriate (subclass) of Configuration. Creating a config initializes some Hadoop
-  // subsystems. Always create a new config, don't reuse yarnConf.
-  override def newConfiguration(conf: SparkConf): Configuration =
-    new YarnConfiguration(super.newConfiguration(conf))
-
-  // Add any user credentials to the job conf which are necessary for running on a secure Hadoop
-  // cluster
-  override def addCredentials(conf: JobConf) {
-    val jobCreds = conf.getCredentials()
-    jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials())
-  }
-
-  override def getCurrentUserCredentials(): Credentials = {
-    UserGroupInformation.getCurrentUser().getCredentials()
-  }
-
-  override def addCurrentUserCredentials(creds: Credentials) {
-    UserGroupInformation.getCurrentUser().addCredentials(creds)
-  }
-
-  override def addSecretKeyToUserCredentials(key: String, secret: String) {
-    val creds = new Credentials()
-    creds.addSecretKey(new Text(key), secret.getBytes(UTF_8))
-    addCurrentUserCredentials(creds)
-  }
-
-  override def getSecretKeyFromUserCredentials(key: String): Array[Byte] = {
-    val credentials = getCurrentUserCredentials()
-    if (credentials != null) credentials.getSecretKey(new Text(key)) else null
-  }
-
-  private[spark] override def startCredentialUpdater(sparkConf: SparkConf): Unit = {
-    credentialUpdater =
-      new ConfigurableCredentialManager(sparkConf, newConfiguration(sparkConf)).credentialUpdater()
-    credentialUpdater.start()
-  }
-
-  private[spark] override def stopCredentialUpdater(): Unit = {
-    if (credentialUpdater != null) {
-      credentialUpdater.stop()
-      credentialUpdater = null
-    }
-  }
-
-  private[spark] def getContainerId: ContainerId = {
-    val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name())
-    ConverterUtils.toContainerId(containerIdString)
-  }
-}
-
-object YarnSparkHadoopUtil {
-  // Additional memory overhead
-  // 10% was arrived at experimentally. In the interest of minimizing memory waste while covering
-  // the common cases. Memory overhead tends to grow with container size.
-
-  val MEMORY_OVERHEAD_FACTOR = 0.10
-  val MEMORY_OVERHEAD_MIN = 384L
-
-  val ANY_HOST = "*"
-
-  val DEFAULT_NUMBER_EXECUTORS = 2
-
-  // All RM requests are issued with same priority : we do not (yet) have any distinction between
-  // request types (like map/reduce in hadoop for example)
-  val RM_REQUEST_PRIORITY = Priority.newInstance(1)
-
-  def get: YarnSparkHadoopUtil = {
-    val yarnMode = java.lang.Boolean.parseBoolean(
-      System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
-    if (!yarnMode) {
-      throw new SparkException("YarnSparkHadoopUtil is not available in non-YARN mode!")
-    }
-    SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil]
-  }
-  /**
-   * Add a path variable to the given environment map.
-   * If the map already contains this key, append the value to the existing value instead.
-   */
-  def addPathToEnvironment(env: HashMap[String, String], key: String, value: String): Unit = {
-    val newValue = if (env.contains(key)) { env(key) + getClassPathSeparator  + value } else value
-    env.put(key, newValue)
-  }
-
-  /**
-   * Set zero or more environment variables specified by the given input string.
-   * The input string is expected to take the form "KEY1=VAL1,KEY2=VAL2,KEY3=VAL3".
-   */
-  def setEnvFromInputString(env: HashMap[String, String], inputString: String): Unit = {
-    if (inputString != null && inputString.length() > 0) {
-      val childEnvs = inputString.split(",")
-      val p = Pattern.compile(environmentVariableRegex)
-      for (cEnv <- childEnvs) {
-        val parts = cEnv.split("=") // split on '='
-        val m = p.matcher(parts(1))
-        val sb = new StringBuffer
-        while (m.find()) {
-          val variable = m.group(1)
-          var replace = ""
-          if (env.get(variable) != None) {
-            replace = env.get(variable).get
-          } else {
-            // if this key is not configured for the child .. get it from the env
-            replace = System.getenv(variable)
-            if (replace == null) {
-            // the env key is note present anywhere .. simply set it
-              replace = ""
-            }
-          }
-          m.appendReplacement(sb, Matcher.quoteReplacement(replace))
-        }
-        m.appendTail(sb)
-        // This treats the environment variable as path variable delimited by `File.pathSeparator`
-        // This is kept for backward compatibility and consistency with Hadoop's behavior
-        addPathToEnvironment(env, parts(0), sb.toString)
-      }
-    }
-  }
-
-  private val environmentVariableRegex: String = {
-    if (Utils.isWindows) {
-      "%([A-Za-z_][A-Za-z0-9_]*?)%"
-    } else {
-      "\\$([A-Za-z_][A-Za-z0-9_]*)"
-    }
-  }
-
-  /**
-   * Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
-   * Not killing the task leaves various aspects of the executor and (to some extent) the jvm in
-   * an inconsistent state.
-   * TODO: If the OOM is not recoverable by rescheduling it on different node, then do
-   * 'something' to fail job ... akin to blacklisting trackers in mapred ?
-   *
-   * The handler if an OOM Exception is thrown by the JVM must be configured on Windows
-   * differently: the 'taskkill' command should be used, whereas Unix-based systems use 'kill'.
-   *
-   * As the JVM interprets both %p and %%p as the same, we can use either of them. However,
-   * some tests on Windows computers suggest, that the JVM only accepts '%%p'.
-   *
-   * Furthermore, the behavior of the character '%' on the Windows command line differs from
-   * the behavior of '%' in a .cmd file: it gets interpreted as an incomplete environment
-   * variable. Windows .cmd files escape a '%' by '%%'. Thus, the correct way of writing
-   * '%%p' in an escaped way is '%%%%p'.
-   */
-  private[yarn] def addOutOfMemoryErrorArgument(javaOpts: ListBuffer[String]): Unit = {
-    if (!javaOpts.exists(_.contains("-XX:OnOutOfMemoryError"))) {
-      if (Utils.isWindows) {
-        javaOpts += escapeForShell("-XX:OnOutOfMemoryError=taskkill /F /PID %%%%p")
-      } else {
-        javaOpts += "-XX:OnOutOfMemoryError='kill %p'"
-      }
-    }
-  }
-
-  /**
-   * Escapes a string for inclusion in a command line executed by Yarn. Yarn executes commands
-   * using either
-   *
-   * (Unix-based) `bash -c "command arg1 arg2"` and that means plain quoting doesn't really work.
-   * The argument is enclosed in single quotes and some key characters are escaped.
-   *
-   * (Windows-based) part of a .cmd file in which case windows escaping for each argument must be
-   * applied. Windows is quite lenient, however it is usually Java that causes trouble, needing to
-   * distinguish between arguments starting with '-' and class names. If arguments are surrounded
-   * by ' java takes the following string as is, hence an argument is mistakenly taken as a class
-   * name which happens to start with a '-'. The way to avoid this, is to surround nothing with
-   * a ', but instead with a ".
-   *
-   * @param arg A single argument.
-   * @return Argument quoted for execution via Yarn's generated shell script.
-   */
-  def escapeForShell(arg: String): String = {
-    if (arg != null) {
-      if (Utils.isWindows) {
-        YarnCommandBuilderUtils.quoteForBatchScript(arg)
-      } else {
-        val escaped = new StringBuilder("'")
-        for (i <- 0 to arg.length() - 1) {
-          arg.charAt(i) match {
-            case '$' => escaped.append("\\$")
-            case '"' => escaped.append("\\\"")
-            case '\'' => escaped.append("'\\''")
-            case c => escaped.append(c)
-          }
-        }
-        escaped.append("'").toString()
-      }
-    } else {
-      arg
-    }
-  }
-
-  // YARN/Hadoop acls are specified as user1,user2 group1,group2
-  // Users and groups are separated by a space and hence we need to pass the acls in same format
-  def getApplicationAclsForYarn(securityMgr: SecurityManager)
-      : Map[ApplicationAccessType, String] = {
-    Map[ApplicationAccessType, String] (
-      ApplicationAccessType.VIEW_APP -> (securityMgr.getViewAcls + " " +
-        securityMgr.getViewAclsGroups),
-      ApplicationAccessType.MODIFY_APP -> (securityMgr.getModifyAcls + " " +
-        securityMgr.getModifyAclsGroups)
-    )
-  }
-
-  /**
-   * Expand environment variable using Yarn API.
-   * If environment.$$() is implemented, return the result of it.
-   * Otherwise, return the result of environment.$()
-   * Note: $$() is added in Hadoop 2.4.
-   */
-  private lazy val expandMethod =
-    Try(classOf[Environment].getMethod("$$"))
-      .getOrElse(classOf[Environment].getMethod("$"))
-
-  def expandEnvironment(environment: Environment): String =
-    expandMethod.invoke(environment).asInstanceOf[String]
-
-  /**
-   * Get class path separator using Yarn API.
-   * If ApplicationConstants.CLASS_PATH_SEPARATOR is implemented, return it.
-   * Otherwise, return File.pathSeparator
-   * Note: CLASS_PATH_SEPARATOR is added in Hadoop 2.4.
-   */
-  private lazy val classPathSeparatorField =
-    Try(classOf[ApplicationConstants].getField("CLASS_PATH_SEPARATOR"))
-      .getOrElse(classOf[File].getField("pathSeparator"))
-
-  def getClassPathSeparator(): String = {
-    classPathSeparatorField.get(null).asInstanceOf[String]
-  }
-
-  /**
-   * Getting the initial target number of executors depends on whether dynamic allocation is
-   * enabled.
-   * If not using dynamic allocation it gets the number of executors requested by the user.
-   */
-  def getInitialTargetExecutorNumber(
-      conf: SparkConf,
-      numExecutors: Int = DEFAULT_NUMBER_EXECUTORS): Int = {
-    if (Utils.isDynamicAllocationEnabled(conf)) {
-      val minNumExecutors = conf.get(DYN_ALLOCATION_MIN_EXECUTORS)
-      val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf)
-      val maxNumExecutors = conf.get(DYN_ALLOCATION_MAX_EXECUTORS)
-      require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors,
-        s"initial executor number $initialNumExecutors must between min executor number " +
-          s"$minNumExecutors and max executor number $maxNumExecutors")
-
-      initialNumExecutors
-    } else {
-      val targetNumExecutors =
-        sys.env.get("SPARK_EXECUTOR_INSTANCES").map(_.toInt).getOrElse(numExecutors)
-      // System property can override environment variable.
-      conf.get(EXECUTOR_INSTANCES).getOrElse(targetNumExecutors)
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
deleted file mode 100644
index 666cb45..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
+++ /dev/null
@@ -1,347 +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.deploy.yarn
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.spark.internal.config.ConfigBuilder
-import org.apache.spark.network.util.ByteUnit
-
-package object config {
-
-  /* Common app configuration. */
-
-  private[spark] val APPLICATION_TAGS = ConfigBuilder("spark.yarn.tags")
-    .doc("Comma-separated list of strings to pass through as YARN application tags appearing " +
-      "in YARN Application Reports, which can be used for filtering when querying YARN.")
-    .stringConf
-    .toSequence
-    .createOptional
-
-  private[spark] val AM_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS =
-    ConfigBuilder("spark.yarn.am.attemptFailuresValidityInterval")
-      .doc("Interval after which AM failures will be considered independent and " +
-        "not accumulate towards the attempt count.")
-      .timeConf(TimeUnit.MILLISECONDS)
-      .createOptional
-
-  private[spark] val AM_PORT =
-    ConfigBuilder("spark.yarn.am.port")
-      .intConf
-      .createWithDefault(0)
-
-  private[spark] val EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS =
-    ConfigBuilder("spark.yarn.executor.failuresValidityInterval")
-      .doc("Interval after which Executor failures will be considered independent and not " +
-        "accumulate towards the attempt count.")
-      .timeConf(TimeUnit.MILLISECONDS)
-      .createOptional
-
-  private[spark] val MAX_APP_ATTEMPTS = ConfigBuilder("spark.yarn.maxAppAttempts")
-    .doc("Maximum number of AM attempts before failing the app.")
-    .intConf
-    .createOptional
-
-  private[spark] val USER_CLASS_PATH_FIRST = ConfigBuilder("spark.yarn.user.classpath.first")
-    .doc("Whether to place user jars in front of Spark's classpath.")
-    .booleanConf
-    .createWithDefault(false)
-
-  private[spark] val GATEWAY_ROOT_PATH = ConfigBuilder("spark.yarn.config.gatewayPath")
-    .doc("Root of configuration paths that is present on gateway nodes, and will be replaced " +
-      "with the corresponding path in cluster machines.")
-    .stringConf
-    .createWithDefault(null)
-
-  private[spark] val REPLACEMENT_ROOT_PATH = ConfigBuilder("spark.yarn.config.replacementPath")
-    .doc(s"Path to use as a replacement for ${GATEWAY_ROOT_PATH.key} when launching processes " +
-      "in the YARN cluster.")
-    .stringConf
-    .createWithDefault(null)
-
-  private[spark] val QUEUE_NAME = ConfigBuilder("spark.yarn.queue")
-    .stringConf
-    .createWithDefault("default")
-
-  private[spark] val HISTORY_SERVER_ADDRESS = ConfigBuilder("spark.yarn.historyServer.address")
-    .stringConf
-    .createOptional
-
-  /* File distribution. */
-
-  private[spark] val SPARK_ARCHIVE = ConfigBuilder("spark.yarn.archive")
-    .doc("Location of archive containing jars files with Spark classes.")
-    .stringConf
-    .createOptional
-
-  private[spark] val SPARK_JARS = ConfigBuilder("spark.yarn.jars")
-    .doc("Location of jars containing Spark classes.")
-    .stringConf
-    .toSequence
-    .createOptional
-
-  private[spark] val ARCHIVES_TO_DISTRIBUTE = ConfigBuilder("spark.yarn.dist.archives")
-    .stringConf
-    .toSequence
-    .createWithDefault(Nil)
-
-  private[spark] val FILES_TO_DISTRIBUTE = ConfigBuilder("spark.yarn.dist.files")
-    .stringConf
-    .toSequence
-    .createWithDefault(Nil)
-
-  private[spark] val JARS_TO_DISTRIBUTE = ConfigBuilder("spark.yarn.dist.jars")
-    .stringConf
-    .toSequence
-    .createWithDefault(Nil)
-
-  private[spark] val PRESERVE_STAGING_FILES = ConfigBuilder("spark.yarn.preserve.staging.files")
-    .doc("Whether to preserve temporary files created by the job in HDFS.")
-    .booleanConf
-    .createWithDefault(false)
-
-  private[spark] val STAGING_FILE_REPLICATION = ConfigBuilder("spark.yarn.submit.file.replication")
-    .doc("Replication factor for files uploaded by Spark to HDFS.")
-    .intConf
-    .createOptional
-
-  private[spark] val STAGING_DIR = ConfigBuilder("spark.yarn.stagingDir")
-    .doc("Staging directory used while submitting applications.")
-    .stringConf
-    .createOptional
-
-  /* Cluster-mode launcher configuration. */
-
-  private[spark] val WAIT_FOR_APP_COMPLETION = ConfigBuilder("spark.yarn.submit.waitAppCompletion")
-    .doc("In cluster mode, whether to wait for the application to finish before exiting the " +
-      "launcher process.")
-    .booleanConf
-    .createWithDefault(true)
-
-  private[spark] val REPORT_INTERVAL = ConfigBuilder("spark.yarn.report.interval")
-    .doc("Interval between reports of the current app status in cluster mode.")
-    .timeConf(TimeUnit.MILLISECONDS)
-    .createWithDefaultString("1s")
-
-  /* Shared Client-mode AM / Driver configuration. */
-
-  private[spark] val AM_MAX_WAIT_TIME = ConfigBuilder("spark.yarn.am.waitTime")
-    .timeConf(TimeUnit.MILLISECONDS)
-    .createWithDefaultString("100s")
-
-  private[spark] val AM_NODE_LABEL_EXPRESSION = ConfigBuilder("spark.yarn.am.nodeLabelExpression")
-    .doc("Node label expression for the AM.")
-    .stringConf
-    .createOptional
-
-  private[spark] val CONTAINER_LAUNCH_MAX_THREADS =
-    ConfigBuilder("spark.yarn.containerLauncherMaxThreads")
-      .intConf
-      .createWithDefault(25)
-
-  private[spark] val MAX_EXECUTOR_FAILURES = ConfigBuilder("spark.yarn.max.executor.failures")
-    .intConf
-    .createOptional
-
-  private[spark] val MAX_REPORTER_THREAD_FAILURES =
-    ConfigBuilder("spark.yarn.scheduler.reporterThread.maxFailures")
-      .intConf
-      .createWithDefault(5)
-
-  private[spark] val RM_HEARTBEAT_INTERVAL =
-    ConfigBuilder("spark.yarn.scheduler.heartbeat.interval-ms")
-      .timeConf(TimeUnit.MILLISECONDS)
-      .createWithDefaultString("3s")
-
-  private[spark] val INITIAL_HEARTBEAT_INTERVAL =
-    ConfigBuilder("spark.yarn.scheduler.initial-allocation.interval")
-      .timeConf(TimeUnit.MILLISECONDS)
-      .createWithDefaultString("200ms")
-
-  private[spark] val SCHEDULER_SERVICES = ConfigBuilder("spark.yarn.services")
-    .doc("A comma-separated list of class names of services to add to the scheduler.")
-    .stringConf
-    .toSequence
-    .createWithDefault(Nil)
-
-  /* Client-mode AM configuration. */
-
-  private[spark] val AM_CORES = ConfigBuilder("spark.yarn.am.cores")
-    .intConf
-    .createWithDefault(1)
-
-  private[spark] val AM_JAVA_OPTIONS = ConfigBuilder("spark.yarn.am.extraJavaOptions")
-    .doc("Extra Java options for the client-mode AM.")
-    .stringConf
-    .createOptional
-
-  private[spark] val AM_LIBRARY_PATH = ConfigBuilder("spark.yarn.am.extraLibraryPath")
-    .doc("Extra native library path for the client-mode AM.")
-    .stringConf
-    .createOptional
-
-  private[spark] val AM_MEMORY_OVERHEAD = ConfigBuilder("spark.yarn.am.memoryOverhead")
-    .bytesConf(ByteUnit.MiB)
-    .createOptional
-
-  private[spark] val AM_MEMORY = ConfigBuilder("spark.yarn.am.memory")
-    .bytesConf(ByteUnit.MiB)
-    .createWithDefaultString("512m")
-
-  /* Driver configuration. */
-
-  private[spark] val DRIVER_CORES = ConfigBuilder("spark.driver.cores")
-    .intConf
-    .createWithDefault(1)
-
-  private[spark] val DRIVER_MEMORY_OVERHEAD = ConfigBuilder("spark.yarn.driver.memoryOverhead")
-    .bytesConf(ByteUnit.MiB)
-    .createOptional
-
-  /* Executor configuration. */
-
-  private[spark] val EXECUTOR_CORES = ConfigBuilder("spark.executor.cores")
-    .intConf
-    .createWithDefault(1)
-
-  private[spark] val EXECUTOR_MEMORY_OVERHEAD = ConfigBuilder("spark.yarn.executor.memoryOverhead")
-    .bytesConf(ByteUnit.MiB)
-    .createOptional
-
-  private[spark] val EXECUTOR_NODE_LABEL_EXPRESSION =
-    ConfigBuilder("spark.yarn.executor.nodeLabelExpression")
-      .doc("Node label expression for executors.")
-      .stringConf
-      .createOptional
-
-  /* Security configuration. */
-
-  private[spark] val CREDENTIAL_FILE_MAX_COUNT =
-    ConfigBuilder("spark.yarn.credentials.file.retention.count")
-      .intConf
-      .createWithDefault(5)
-
-  private[spark] val CREDENTIALS_FILE_MAX_RETENTION =
-    ConfigBuilder("spark.yarn.credentials.file.retention.days")
-      .intConf
-      .createWithDefault(5)
-
-  private[spark] val NAMENODES_TO_ACCESS = ConfigBuilder("spark.yarn.access.namenodes")
-    .doc("Extra NameNode URLs for which to request delegation tokens. The NameNode that hosts " +
-      "fs.defaultFS does not need to be listed here.")
-    .stringConf
-    .toSequence
-    .createWithDefault(Nil)
-
-  /* Rolled log aggregation configuration. */
-
-  private[spark] val ROLLED_LOG_INCLUDE_PATTERN =
-    ConfigBuilder("spark.yarn.rolledLog.includePattern")
-      .doc("Java Regex to filter the log files which match the defined include pattern and those " +
-        "log files will be aggregated in a rolling fashion.")
-      .stringConf
-      .createOptional
-
-  private[spark] val ROLLED_LOG_EXCLUDE_PATTERN =
-    ConfigBuilder("spark.yarn.rolledLog.excludePattern")
-      .doc("Java Regex to filter the log files which match the defined exclude pattern and those " +
-        "log files will not be aggregated in a rolling fashion.")
-      .stringConf
-      .createOptional
-
-  /* Private configs. */
-
-  private[spark] val CREDENTIALS_FILE_PATH = ConfigBuilder("spark.yarn.credentials.file")
-    .internal()
-    .stringConf
-    .createWithDefault(null)
-
-  // Internal config to propagate the location of the user's jar to the driver/executors
-  private[spark] val APP_JAR = ConfigBuilder("spark.yarn.user.jar")
-    .internal()
-    .stringConf
-    .createOptional
-
-  // Internal config to propagate the locations of any extra jars to add to the classpath
-  // of the executors
-  private[spark] val SECONDARY_JARS = ConfigBuilder("spark.yarn.secondary.jars")
-    .internal()
-    .stringConf
-    .toSequence
-    .createOptional
-
-  /* Configuration and cached file propagation. */
-
-  private[spark] val CACHED_FILES = ConfigBuilder("spark.yarn.cache.filenames")
-    .internal()
-    .stringConf
-    .toSequence
-    .createWithDefault(Nil)
-
-  private[spark] val CACHED_FILES_SIZES = ConfigBuilder("spark.yarn.cache.sizes")
-    .internal()
-    .longConf
-    .toSequence
-    .createWithDefault(Nil)
-
-  private[spark] val CACHED_FILES_TIMESTAMPS = ConfigBuilder("spark.yarn.cache.timestamps")
-    .internal()
-    .longConf
-    .toSequence
-    .createWithDefault(Nil)
-
-  private[spark] val CACHED_FILES_VISIBILITIES = ConfigBuilder("spark.yarn.cache.visibilities")
-    .internal()
-    .stringConf
-    .toSequence
-    .createWithDefault(Nil)
-
-  // Either "file" or "archive", for each file.
-  private[spark] val CACHED_FILES_TYPES = ConfigBuilder("spark.yarn.cache.types")
-    .internal()
-    .stringConf
-    .toSequence
-    .createWithDefault(Nil)
-
-  // The location of the conf archive in HDFS.
-  private[spark] val CACHED_CONF_ARCHIVE = ConfigBuilder("spark.yarn.cache.confArchive")
-    .internal()
-    .stringConf
-    .createOptional
-
-  private[spark] val CREDENTIALS_RENEWAL_TIME = ConfigBuilder("spark.yarn.credentials.renewalTime")
-    .internal()
-    .timeConf(TimeUnit.MILLISECONDS)
-    .createWithDefault(Long.MaxValue)
-
-  private[spark] val CREDENTIALS_UPDATE_TIME = ConfigBuilder("spark.yarn.credentials.updateTime")
-    .internal()
-    .timeConf(TimeUnit.MILLISECONDS)
-    .createWithDefault(Long.MaxValue)
-
-  // The list of cache-related config entries. This is used by Client and the AM to clean
-  // up the environment so that these settings do not appear on the web UI.
-  private[yarn] val CACHE_CONFIGS = Seq(
-    CACHED_FILES,
-    CACHED_FILES_SIZES,
-    CACHED_FILES_TIMESTAMPS,
-    CACHED_FILES_VISIBILITIES,
-    CACHED_FILES_TYPES,
-    CACHED_CONF_ARCHIVE)
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala
deleted file mode 100644
index 7e76f40..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala
+++ /dev/null
@@ -1,235 +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.deploy.yarn.security
-
-import java.security.PrivilegedExceptionAction
-import java.util.concurrent.{Executors, TimeUnit}
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileSystem, Path}
-import org.apache.hadoop.security.UserGroupInformation
-
-import org.apache.spark.SparkConf
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.internal.Logging
-import org.apache.spark.internal.config._
-import org.apache.spark.util.ThreadUtils
-
-/**
- * The following methods are primarily meant to make sure long-running apps like Spark
- * Streaming apps can run without interruption while accessing secured services. The
- * scheduleLoginFromKeytab method is called on the AM to get the new credentials.
- * This method wakes up a thread that logs into the KDC
- * once 75% of the renewal interval of the original credentials used for the container
- * has elapsed. It then obtains new credentials and writes them to HDFS in a
- * pre-specified location - the prefix of which is specified in the sparkConf by
- * spark.yarn.credentials.file (so the file(s) would be named c-timestamp1-1, c-timestamp2-2 etc.
- * - each update goes to a new file, with a monotonically increasing suffix), also the
- * timestamp1, timestamp2 here indicates the time of next update for CredentialUpdater.
- * After this, the credentials are renewed once 75% of the new tokens renewal interval has elapsed.
- *
- * On the executor and driver (yarn client mode) side, the updateCredentialsIfRequired method is
- * called once 80% of the validity of the original credentials has elapsed. At that time the
- * executor finds the credentials file with the latest timestamp and checks if it has read those
- * credentials before (by keeping track of the suffix of the last file it read). If a new file has
- * appeared, it will read the credentials and update the currently running UGI with it. This
- * process happens again once 80% of the validity of this has expired.
- */
-private[yarn] class AMCredentialRenewer(
-    sparkConf: SparkConf,
-    hadoopConf: Configuration,
-    credentialManager: ConfigurableCredentialManager) extends Logging {
-
-  private var lastCredentialsFileSuffix = 0
-
-  private val credentialRenewer =
-    Executors.newSingleThreadScheduledExecutor(
-      ThreadUtils.namedThreadFactory("Credential Refresh Thread"))
-
-  private val hadoopUtil = YarnSparkHadoopUtil.get
-
-  private val credentialsFile = sparkConf.get(CREDENTIALS_FILE_PATH)
-  private val daysToKeepFiles = sparkConf.get(CREDENTIALS_FILE_MAX_RETENTION)
-  private val numFilesToKeep = sparkConf.get(CREDENTIAL_FILE_MAX_COUNT)
-  private val freshHadoopConf =
-    hadoopUtil.getConfBypassingFSCache(hadoopConf, new Path(credentialsFile).toUri.getScheme)
-
-  @volatile private var timeOfNextRenewal = sparkConf.get(CREDENTIALS_RENEWAL_TIME)
-
-  /**
-   * Schedule a login from the keytab and principal set using the --principal and --keytab
-   * arguments to spark-submit. This login happens only when the credentials of the current user
-   * are about to expire. This method reads spark.yarn.principal and spark.yarn.keytab from
-   * SparkConf to do the login. This method is a no-op in non-YARN mode.
-   *
-   */
-  private[spark] def scheduleLoginFromKeytab(): Unit = {
-    val principal = sparkConf.get(PRINCIPAL).get
-    val keytab = sparkConf.get(KEYTAB).get
-
-    /**
-     * Schedule re-login and creation of new credentials. If credentials have already expired, this
-     * method will synchronously create new ones.
-     */
-    def scheduleRenewal(runnable: Runnable): Unit = {
-      // Run now!
-      val remainingTime = timeOfNextRenewal - System.currentTimeMillis()
-      if (remainingTime <= 0) {
-        logInfo("Credentials have expired, creating new ones now.")
-        runnable.run()
-      } else {
-        logInfo(s"Scheduling login from keytab in $remainingTime millis.")
-        credentialRenewer.schedule(runnable, remainingTime, TimeUnit.MILLISECONDS)
-      }
-    }
-
-    // This thread periodically runs on the AM to update the credentials on HDFS.
-    val credentialRenewerRunnable =
-      new Runnable {
-        override def run(): Unit = {
-          try {
-            writeNewCredentialsToHDFS(principal, keytab)
-            cleanupOldFiles()
-          } catch {
-            case e: Exception =>
-              // Log the error and try to write new tokens back in an hour
-              logWarning("Failed to write out new credentials to HDFS, will try again in an " +
-                "hour! If this happens too often tasks will fail.", e)
-              credentialRenewer.schedule(this, 1, TimeUnit.HOURS)
-              return
-          }
-          scheduleRenewal(this)
-        }
-      }
-    // Schedule update of credentials. This handles the case of updating the credentials right now
-    // as well, since the renewal interval will be 0, and the thread will get scheduled
-    // immediately.
-    scheduleRenewal(credentialRenewerRunnable)
-  }
-
-  // Keeps only files that are newer than daysToKeepFiles days, and deletes everything else. At
-  // least numFilesToKeep files are kept for safety
-  private def cleanupOldFiles(): Unit = {
-    import scala.concurrent.duration._
-    try {
-      val remoteFs = FileSystem.get(freshHadoopConf)
-      val credentialsPath = new Path(credentialsFile)
-      val thresholdTime = System.currentTimeMillis() - (daysToKeepFiles.days).toMillis
-      hadoopUtil.listFilesSorted(
-        remoteFs, credentialsPath.getParent,
-        credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
-        .dropRight(numFilesToKeep)
-        .takeWhile(_.getModificationTime < thresholdTime)
-        .foreach(x => remoteFs.delete(x.getPath, true))
-    } catch {
-      // Such errors are not fatal, so don't throw. Make sure they are logged though
-      case e: Exception =>
-        logWarning("Error while attempting to cleanup old credentials. If you are seeing many " +
-          "such warnings there may be an issue with your HDFS cluster.", e)
-    }
-  }
-
-  private def writeNewCredentialsToHDFS(principal: String, keytab: String): Unit = {
-    // Keytab is copied by YARN to the working directory of the AM, so full path is
-    // not needed.
-
-    // HACK:
-    // HDFS will not issue new delegation tokens, if the Credentials object
-    // passed in already has tokens for that FS even if the tokens are expired (it really only
-    // checks if there are tokens for the service, and not if they are valid). So the only real
-    // way to get new tokens is to make sure a different Credentials object is used each time to
-    // get new tokens and then the new tokens are copied over the current user's Credentials.
-    // So:
-    // - we login as a different user and get the UGI
-    // - use that UGI to get the tokens (see doAs block below)
-    // - copy the tokens over to the current user's credentials (this will overwrite the tokens
-    // in the current user's Credentials object for this FS).
-    // The login to KDC happens each time new tokens are required, but this is rare enough to not
-    // have to worry about (like once every day or so). This makes this code clearer than having
-    // to login and then relogin every time (the HDFS API may not relogin since we don't use this
-    // UGI directly for HDFS communication.
-    logInfo(s"Attempting to login to KDC using principal: $principal")
-    val keytabLoggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab)
-    logInfo("Successfully logged into KDC.")
-    val tempCreds = keytabLoggedInUGI.getCredentials
-    val credentialsPath = new Path(credentialsFile)
-    val dst = credentialsPath.getParent
-    var nearestNextRenewalTime = Long.MaxValue
-    keytabLoggedInUGI.doAs(new PrivilegedExceptionAction[Void] {
-      // Get a copy of the credentials
-      override def run(): Void = {
-        nearestNextRenewalTime = credentialManager.obtainCredentials(freshHadoopConf, tempCreds)
-        null
-      }
-    })
-
-    val currTime = System.currentTimeMillis()
-    val timeOfNextUpdate = if (nearestNextRenewalTime <= currTime) {
-      // If next renewal time is earlier than current time, we set next renewal time to current
-      // time, this will trigger next renewal immediately. Also set next update time to current
-      // time. There still has a gap between token renewal and update will potentially introduce
-      // issue.
-      logWarning(s"Next credential renewal time ($nearestNextRenewalTime) is earlier than " +
-        s"current time ($currTime), which is unexpected, please check your credential renewal " +
-        "related configurations in the target services.")
-      timeOfNextRenewal = currTime
-      currTime
-    } else {
-      // Next valid renewal time is about 75% of credential renewal time, and update time is
-      // slightly later than valid renewal time (80% of renewal time).
-      timeOfNextRenewal = ((nearestNextRenewalTime - currTime) * 0.75 + currTime).toLong
-      ((nearestNextRenewalTime - currTime) * 0.8 + currTime).toLong
-    }
-
-    // Add the temp credentials back to the original ones.
-    UserGroupInformation.getCurrentUser.addCredentials(tempCreds)
-    val remoteFs = FileSystem.get(freshHadoopConf)
-    // If lastCredentialsFileSuffix is 0, then the AM is either started or restarted. If the AM
-    // was restarted, then the lastCredentialsFileSuffix might be > 0, so find the newest file
-    // and update the lastCredentialsFileSuffix.
-    if (lastCredentialsFileSuffix == 0) {
-      hadoopUtil.listFilesSorted(
-        remoteFs, credentialsPath.getParent,
-        credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
-        .lastOption.foreach { status =>
-        lastCredentialsFileSuffix = hadoopUtil.getSuffixForCredentialsPath(status.getPath)
-      }
-    }
-    val nextSuffix = lastCredentialsFileSuffix + 1
-
-    val tokenPathStr =
-      credentialsFile + SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM +
-        timeOfNextUpdate.toLong.toString + SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM +
-          nextSuffix
-    val tokenPath = new Path(tokenPathStr)
-    val tempTokenPath = new Path(tokenPathStr + SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
-
-    logInfo("Writing out delegation tokens to " + tempTokenPath.toString)
-    val credentials = UserGroupInformation.getCurrentUser.getCredentials
-    credentials.writeTokenStorageFile(tempTokenPath, freshHadoopConf)
-    logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr")
-    remoteFs.rename(tempTokenPath, tokenPath)
-    logInfo("Delegation token file rename complete.")
-    lastCredentialsFileSuffix = nextSuffix
-  }
-
-  def stop(): Unit = {
-    credentialRenewer.shutdown()
-  }
-}


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


[07/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
new file mode 100644
index 0000000..7fbbe12
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
@@ -0,0 +1,213 @@
+/*
+ * 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 java.io.{File, IOException}
+import java.nio.charset.StandardCharsets
+
+import com.google.common.io.{ByteStreams, Files}
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.yarn.api.ApplicationConstants
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.scalatest.Matchers
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.{ResetSystemProperties, Utils}
+
+class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging
+  with ResetSystemProperties {
+
+  val hasBash =
+    try {
+      val exitCode = Runtime.getRuntime().exec(Array("bash", "--version")).waitFor()
+      exitCode == 0
+    } catch {
+      case e: IOException =>
+        false
+    }
+
+  if (!hasBash) {
+    logWarning("Cannot execute bash, skipping bash tests.")
+  }
+
+  def bashTest(name: String)(fn: => Unit): Unit =
+    if (hasBash) test(name)(fn) else ignore(name)(fn)
+
+  bashTest("shell script escaping") {
+    val scriptFile = File.createTempFile("script.", ".sh", Utils.createTempDir())
+    val args = Array("arg1", "${arg.2}", "\"arg3\"", "'arg4'", "$arg5", "\\arg6")
+    try {
+      val argLine = args.map(a => YarnSparkHadoopUtil.escapeForShell(a)).mkString(" ")
+      Files.write(("bash -c \"echo " + argLine + "\"").getBytes(StandardCharsets.UTF_8), scriptFile)
+      scriptFile.setExecutable(true)
+
+      val proc = Runtime.getRuntime().exec(Array(scriptFile.getAbsolutePath()))
+      val out = new String(ByteStreams.toByteArray(proc.getInputStream())).trim()
+      val err = new String(ByteStreams.toByteArray(proc.getErrorStream()))
+      val exitCode = proc.waitFor()
+      exitCode should be (0)
+      out should be (args.mkString(" "))
+    } finally {
+      scriptFile.delete()
+    }
+  }
+
+  test("Yarn configuration override") {
+    val key = "yarn.nodemanager.hostname"
+    val default = new YarnConfiguration()
+
+    val sparkConf = new SparkConf()
+      .set("spark.hadoop." + key, "someHostName")
+    val yarnConf = new YarnSparkHadoopUtil().newConfiguration(sparkConf)
+
+    yarnConf.getClass() should be (classOf[YarnConfiguration])
+    yarnConf.get(key) should not be default.get(key)
+  }
+
+
+  test("test getApplicationAclsForYarn acls on") {
+
+    // spark acls on, just pick up default user
+    val sparkConf = new SparkConf()
+    sparkConf.set("spark.acls.enable", "true")
+
+    val securityMgr = new SecurityManager(sparkConf)
+    val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)
+
+    val viewAcls = acls.get(ApplicationAccessType.VIEW_APP)
+    val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP)
+
+    viewAcls match {
+      case Some(vacls) =>
+        val aclSet = vacls.split(',').map(_.trim).toSet
+        assert(aclSet.contains(System.getProperty("user.name", "invalid")))
+      case None =>
+        fail()
+    }
+    modifyAcls match {
+      case Some(macls) =>
+        val aclSet = macls.split(',').map(_.trim).toSet
+        assert(aclSet.contains(System.getProperty("user.name", "invalid")))
+      case None =>
+        fail()
+    }
+  }
+
+  test("test getApplicationAclsForYarn acls on and specify users") {
+
+    // default spark acls are on and specify acls
+    val sparkConf = new SparkConf()
+    sparkConf.set("spark.acls.enable", "true")
+    sparkConf.set("spark.ui.view.acls", "user1,user2")
+    sparkConf.set("spark.modify.acls", "user3,user4")
+
+    val securityMgr = new SecurityManager(sparkConf)
+    val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)
+
+    val viewAcls = acls.get(ApplicationAccessType.VIEW_APP)
+    val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP)
+
+    viewAcls match {
+      case Some(vacls) =>
+        val aclSet = vacls.split(',').map(_.trim).toSet
+        assert(aclSet.contains("user1"))
+        assert(aclSet.contains("user2"))
+        assert(aclSet.contains(System.getProperty("user.name", "invalid")))
+      case None =>
+        fail()
+    }
+    modifyAcls match {
+      case Some(macls) =>
+        val aclSet = macls.split(',').map(_.trim).toSet
+        assert(aclSet.contains("user3"))
+        assert(aclSet.contains("user4"))
+        assert(aclSet.contains(System.getProperty("user.name", "invalid")))
+      case None =>
+        fail()
+    }
+
+  }
+
+  test("test expandEnvironment result") {
+    val target = Environment.PWD
+    if (classOf[Environment].getMethods().exists(_.getName == "$$")) {
+      YarnSparkHadoopUtil.expandEnvironment(target) should be ("{{" + target + "}}")
+    } else if (Utils.isWindows) {
+      YarnSparkHadoopUtil.expandEnvironment(target) should be ("%" + target + "%")
+    } else {
+      YarnSparkHadoopUtil.expandEnvironment(target) should be ("$" + target)
+    }
+
+  }
+
+  test("test getClassPathSeparator result") {
+    if (classOf[ApplicationConstants].getFields().exists(_.getName == "CLASS_PATH_SEPARATOR")) {
+      YarnSparkHadoopUtil.getClassPathSeparator() should be ("<CPS>")
+    } else if (Utils.isWindows) {
+      YarnSparkHadoopUtil.getClassPathSeparator() should be (";")
+    } else {
+      YarnSparkHadoopUtil.getClassPathSeparator() should be (":")
+    }
+  }
+
+  test("check different hadoop utils based on env variable") {
+    try {
+      System.setProperty("SPARK_YARN_MODE", "true")
+      assert(SparkHadoopUtil.get.getClass === classOf[YarnSparkHadoopUtil])
+      System.setProperty("SPARK_YARN_MODE", "false")
+      assert(SparkHadoopUtil.get.getClass === classOf[SparkHadoopUtil])
+    } finally {
+      System.clearProperty("SPARK_YARN_MODE")
+    }
+  }
+
+
+
+  // This test needs to live here because it depends on isYarnMode returning true, which can only
+  // happen in the YARN module.
+  test("security manager token generation") {
+    try {
+      System.setProperty("SPARK_YARN_MODE", "true")
+      val initial = SparkHadoopUtil.get
+        .getSecretKeyFromUserCredentials(SecurityManager.SECRET_LOOKUP_KEY)
+      assert(initial === null || initial.length === 0)
+
+      val conf = new SparkConf()
+        .set(SecurityManager.SPARK_AUTH_CONF, "true")
+        .set(SecurityManager.SPARK_AUTH_SECRET_CONF, "unused")
+      val sm = new SecurityManager(conf)
+
+      val generated = SparkHadoopUtil.get
+        .getSecretKeyFromUserCredentials(SecurityManager.SECRET_LOOKUP_KEY)
+      assert(generated != null)
+      val genString = new Text(generated).toString()
+      assert(genString != "unused")
+      assert(sm.getSecretKey() === genString)
+    } finally {
+      // removeSecretKey() was only added in Hadoop 2.6, so instead we just set the secret
+      // to an empty string.
+      SparkHadoopUtil.get.addSecretKeyToUserCredentials(SecurityManager.SECRET_LOOKUP_KEY, "")
+      System.clearProperty("SPARK_YARN_MODE")
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala
new file mode 100644
index 0000000..db4619e
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala
@@ -0,0 +1,150 @@
+/*
+ * 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.security
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.security.Credentials
+import org.apache.hadoop.security.token.Token
+import org.scalatest.{BeforeAndAfter, Matchers}
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.yarn.config._
+
+class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfter {
+  private var credentialManager: ConfigurableCredentialManager = null
+  private var sparkConf: SparkConf = null
+  private var hadoopConf: Configuration = null
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    sparkConf = new SparkConf()
+    hadoopConf = new Configuration()
+    System.setProperty("SPARK_YARN_MODE", "true")
+  }
+
+  override def afterAll(): Unit = {
+    System.clearProperty("SPARK_YARN_MODE")
+
+    super.afterAll()
+  }
+
+  test("Correctly load default credential providers") {
+    credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
+
+    credentialManager.getServiceCredentialProvider("hdfs") should not be (None)
+    credentialManager.getServiceCredentialProvider("hbase") should not be (None)
+    credentialManager.getServiceCredentialProvider("hive") should not be (None)
+  }
+
+  test("disable hive credential provider") {
+    sparkConf.set("spark.yarn.security.credentials.hive.enabled", "false")
+    credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
+
+    credentialManager.getServiceCredentialProvider("hdfs") should not be (None)
+    credentialManager.getServiceCredentialProvider("hbase") should not be (None)
+    credentialManager.getServiceCredentialProvider("hive") should be (None)
+  }
+
+  test("using deprecated configurations") {
+    sparkConf.set("spark.yarn.security.tokens.hdfs.enabled", "false")
+    sparkConf.set("spark.yarn.security.tokens.hive.enabled", "false")
+    credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
+
+    credentialManager.getServiceCredentialProvider("hdfs") should be (None)
+    credentialManager.getServiceCredentialProvider("hive") should be (None)
+    credentialManager.getServiceCredentialProvider("test") should not be (None)
+    credentialManager.getServiceCredentialProvider("hbase") should not be (None)
+  }
+
+  test("verify obtaining credentials from provider") {
+    credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
+    val creds = new Credentials()
+
+    // Tokens can only be obtained from TestTokenProvider, for hdfs, hbase and hive tokens cannot
+    // be obtained.
+    credentialManager.obtainCredentials(hadoopConf, creds)
+    val tokens = creds.getAllTokens
+    tokens.size() should be (1)
+    tokens.iterator().next().getService should be (new Text("test"))
+  }
+
+  test("verify getting credential renewal info") {
+    credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
+    val creds = new Credentials()
+
+    val testCredentialProvider = credentialManager.getServiceCredentialProvider("test").get
+      .asInstanceOf[TestCredentialProvider]
+    // Only TestTokenProvider can get the time of next token renewal
+    val nextRenewal = credentialManager.obtainCredentials(hadoopConf, creds)
+    nextRenewal should be (testCredentialProvider.timeOfNextTokenRenewal)
+  }
+
+  test("obtain tokens For HiveMetastore") {
+    val hadoopConf = new Configuration()
+    hadoopConf.set("hive.metastore.kerberos.principal", "bob")
+    // thrift picks up on port 0 and bails out, without trying to talk to endpoint
+    hadoopConf.set("hive.metastore.uris", "http://localhost:0")
+
+    val hiveCredentialProvider = new HiveCredentialProvider()
+    val credentials = new Credentials()
+    hiveCredentialProvider.obtainCredentials(hadoopConf, sparkConf, credentials)
+
+    credentials.getAllTokens.size() should be (0)
+  }
+
+  test("Obtain tokens For HBase") {
+    val hadoopConf = new Configuration()
+    hadoopConf.set("hbase.security.authentication", "kerberos")
+
+    val hbaseTokenProvider = new HBaseCredentialProvider()
+    val creds = new Credentials()
+    hbaseTokenProvider.obtainCredentials(hadoopConf, sparkConf, creds)
+
+    creds.getAllTokens.size should be (0)
+  }
+}
+
+class TestCredentialProvider extends ServiceCredentialProvider {
+  val tokenRenewalInterval = 86400 * 1000L
+  var timeOfNextTokenRenewal = 0L
+
+  override def serviceName: String = "test"
+
+  override def credentialsRequired(conf: Configuration): Boolean = true
+
+  override def obtainCredentials(
+      hadoopConf: Configuration,
+      sparkConf: SparkConf,
+      creds: Credentials): Option[Long] = {
+    if (creds == null) {
+      // Guard out other unit test failures.
+      return None
+    }
+
+    val emptyToken = new Token()
+    emptyToken.setService(new Text("test"))
+    creds.addToken(emptyToken.getService, emptyToken)
+
+    val currTime = System.currentTimeMillis()
+    timeOfNextTokenRenewal = (currTime - currTime % tokenRenewalInterval) + tokenRenewalInterval
+
+    Some(timeOfNextTokenRenewal)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProviderSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProviderSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProviderSuite.scala
new file mode 100644
index 0000000..7b2da3f
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProviderSuite.scala
@@ -0,0 +1,71 @@
+/*
+ * 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.security
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.scalatest.{Matchers, PrivateMethodTester}
+
+import org.apache.spark.{SparkConf, SparkException, SparkFunSuite}
+
+class HDFSCredentialProviderSuite
+    extends SparkFunSuite
+    with PrivateMethodTester
+    with Matchers {
+  private val _getTokenRenewer = PrivateMethod[String]('getTokenRenewer)
+
+  private def getTokenRenewer(
+      hdfsCredentialProvider: HDFSCredentialProvider, conf: Configuration): String = {
+    hdfsCredentialProvider invokePrivate _getTokenRenewer(conf)
+  }
+
+  private var hdfsCredentialProvider: HDFSCredentialProvider = null
+
+  override def beforeAll() {
+    super.beforeAll()
+
+    if (hdfsCredentialProvider == null) {
+      hdfsCredentialProvider = new HDFSCredentialProvider()
+    }
+  }
+
+  override def afterAll() {
+    if (hdfsCredentialProvider != null) {
+      hdfsCredentialProvider = null
+    }
+
+    super.afterAll()
+  }
+
+  test("check token renewer") {
+    val hadoopConf = new Configuration()
+    hadoopConf.set("yarn.resourcemanager.address", "myrm:8033")
+    hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM")
+    val renewer = getTokenRenewer(hdfsCredentialProvider, hadoopConf)
+    renewer should be ("yarn/myrm:8032@SPARKTEST.COM")
+  }
+
+  test("check token renewer default") {
+    val hadoopConf = new Configuration()
+    val caught =
+      intercept[SparkException] {
+        getTokenRenewer(hdfsCredentialProvider, hadoopConf)
+      }
+    assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer")
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala
new file mode 100644
index 0000000..da9e8e2
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala
@@ -0,0 +1,36 @@
+/*
+ * 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.launcher
+
+import java.util.{List => JList, Map => JMap}
+
+/**
+ * Exposes AbstractCommandBuilder to the YARN tests, so that they can build classpaths the same
+ * way other cluster managers do.
+ */
+private[spark] class TestClasspathBuilder extends AbstractCommandBuilder {
+
+  childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, sys.props("spark.test.home"))
+
+  override def buildClassPath(extraCp: String): JList[String] = super.buildClassPath(extraCp)
+
+  /** Not used by the YARN tests. */
+  override def buildCommand(env: JMap[String, String]): JList[String] =
+    throw new UnsupportedOperationException()
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala
new file mode 100644
index 0000000..1fed256
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala
@@ -0,0 +1,70 @@
+/*
+ * 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.network.shuffle
+
+import java.io.File
+import java.util.concurrent.ConcurrentMap
+
+import org.apache.hadoop.yarn.api.records.ApplicationId
+import org.fusesource.leveldbjni.JniDBFactory
+import org.iq80.leveldb.{DB, Options}
+
+import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo
+
+/**
+ * just a cheat to get package-visible members in tests
+ */
+object ShuffleTestAccessor {
+
+  def getBlockResolver(handler: ExternalShuffleBlockHandler): ExternalShuffleBlockResolver = {
+    handler.blockManager
+  }
+
+  def getExecutorInfo(
+      appId: ApplicationId,
+      execId: String,
+      resolver: ExternalShuffleBlockResolver
+  ): Option[ExecutorShuffleInfo] = {
+    val id = new AppExecId(appId.toString, execId)
+    Option(resolver.executors.get(id))
+  }
+
+  def registeredExecutorFile(resolver: ExternalShuffleBlockResolver): File = {
+    resolver.registeredExecutorFile
+  }
+
+  def shuffleServiceLevelDB(resolver: ExternalShuffleBlockResolver): DB = {
+    resolver.db
+  }
+
+  def reloadRegisteredExecutors(
+    file: File): ConcurrentMap[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo] = {
+    val options: Options = new Options
+    options.createIfMissing(true)
+    val factory = new JniDBFactory
+    val db = factory.open(file, options)
+    val result = ExternalShuffleBlockResolver.reloadRegisteredExecutors(db)
+    db.close()
+    result
+  }
+
+  def reloadRegisteredExecutors(
+      db: DB): ConcurrentMap[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo] = {
+    ExternalShuffleBlockResolver.reloadRegisteredExecutors(db)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala
new file mode 100644
index 0000000..a58784f
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala
@@ -0,0 +1,372 @@
+/*
+ * 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.network.yarn
+
+import java.io.{DataOutputStream, File, FileOutputStream, IOException}
+import java.nio.ByteBuffer
+import java.nio.file.Files
+import java.nio.file.attribute.PosixFilePermission._
+import java.util.EnumSet
+
+import scala.annotation.tailrec
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.service.ServiceStateException
+import org.apache.hadoop.yarn.api.records.ApplicationId
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.server.api.{ApplicationInitializationContext, ApplicationTerminationContext}
+import org.scalatest.{BeforeAndAfterEach, Matchers}
+import org.scalatest.concurrent.Eventually._
+
+import org.apache.spark.SecurityManager
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.network.shuffle.ShuffleTestAccessor
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo
+import org.apache.spark.util.Utils
+
+class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach {
+  private[yarn] var yarnConfig: YarnConfiguration = null
+  private[yarn] val SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"
+
+  override def beforeEach(): Unit = {
+    super.beforeEach()
+    yarnConfig = new YarnConfiguration()
+    yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle")
+    yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"),
+      classOf[YarnShuffleService].getCanonicalName)
+    yarnConfig.setInt("spark.shuffle.service.port", 0)
+    yarnConfig.setBoolean(YarnShuffleService.STOP_ON_FAILURE_KEY, true)
+    val localDir = Utils.createTempDir()
+    yarnConfig.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath)
+  }
+
+  var s1: YarnShuffleService = null
+  var s2: YarnShuffleService = null
+  var s3: YarnShuffleService = null
+
+  override def afterEach(): Unit = {
+    try {
+      if (s1 != null) {
+        s1.stop()
+        s1 = null
+      }
+      if (s2 != null) {
+        s2.stop()
+        s2 = null
+      }
+      if (s3 != null) {
+        s3.stop()
+        s3 = null
+      }
+    } finally {
+      super.afterEach()
+    }
+  }
+
+  test("executor state kept across NM restart") {
+    s1 = new YarnShuffleService
+    // set auth to true to test the secrets recovery
+    yarnConfig.setBoolean(SecurityManager.SPARK_AUTH_CONF, true)
+    s1.init(yarnConfig)
+    val app1Id = ApplicationId.newInstance(0, 1)
+    val app1Data = makeAppInfo("user", app1Id)
+    s1.initializeApplication(app1Data)
+    val app2Id = ApplicationId.newInstance(0, 2)
+    val app2Data = makeAppInfo("user", app2Id)
+    s1.initializeApplication(app2Data)
+
+    val execStateFile = s1.registeredExecutorFile
+    execStateFile should not be (null)
+    val secretsFile = s1.secretsFile
+    secretsFile should not be (null)
+    val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER)
+    val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER)
+
+    val blockHandler = s1.blockHandler
+    val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler)
+    ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile)
+
+    blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1)
+    blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2)
+    ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", blockResolver) should
+      be (Some(shuffleInfo1))
+    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", blockResolver) should
+      be (Some(shuffleInfo2))
+
+    if (!execStateFile.exists()) {
+      @tailrec def findExistingParent(file: File): File = {
+        if (file == null) file
+        else if (file.exists()) file
+        else findExistingParent(file.getParentFile())
+      }
+      val existingParent = findExistingParent(execStateFile)
+      assert(false, s"$execStateFile does not exist -- closest existing parent is $existingParent")
+    }
+    assert(execStateFile.exists(), s"$execStateFile did not exist")
+
+    // now we pretend the shuffle service goes down, and comes back up
+    s1.stop()
+    s2 = new YarnShuffleService
+    s2.init(yarnConfig)
+    s2.secretsFile should be (secretsFile)
+    s2.registeredExecutorFile should be (execStateFile)
+
+    val handler2 = s2.blockHandler
+    val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2)
+
+    // now we reinitialize only one of the apps, and expect yarn to tell us that app2 was stopped
+    // during the restart
+    s2.initializeApplication(app1Data)
+    s2.stopApplication(new ApplicationTerminationContext(app2Id))
+    ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (Some(shuffleInfo1))
+    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None)
+
+    // Act like the NM restarts one more time
+    s2.stop()
+    s3 = new YarnShuffleService
+    s3.init(yarnConfig)
+    s3.registeredExecutorFile should be (execStateFile)
+    s3.secretsFile should be (secretsFile)
+
+    val handler3 = s3.blockHandler
+    val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3)
+
+    // app1 is still running
+    s3.initializeApplication(app1Data)
+    ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver3) should be (Some(shuffleInfo1))
+    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (None)
+    s3.stop()
+  }
+
+  test("removed applications should not be in registered executor file") {
+    s1 = new YarnShuffleService
+    yarnConfig.setBoolean(SecurityManager.SPARK_AUTH_CONF, false)
+    s1.init(yarnConfig)
+    val secretsFile = s1.secretsFile
+    secretsFile should be (null)
+    val app1Id = ApplicationId.newInstance(0, 1)
+    val app1Data = makeAppInfo("user", app1Id)
+    s1.initializeApplication(app1Data)
+    val app2Id = ApplicationId.newInstance(0, 2)
+    val app2Data = makeAppInfo("user", app2Id)
+    s1.initializeApplication(app2Data)
+
+    val execStateFile = s1.registeredExecutorFile
+    execStateFile should not be (null)
+    val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER)
+    val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER)
+
+    val blockHandler = s1.blockHandler
+    val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler)
+    ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile)
+
+    blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1)
+    blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2)
+
+    val db = ShuffleTestAccessor.shuffleServiceLevelDB(blockResolver)
+    ShuffleTestAccessor.reloadRegisteredExecutors(db) should not be empty
+
+    s1.stopApplication(new ApplicationTerminationContext(app1Id))
+    ShuffleTestAccessor.reloadRegisteredExecutors(db) should not be empty
+    s1.stopApplication(new ApplicationTerminationContext(app2Id))
+    ShuffleTestAccessor.reloadRegisteredExecutors(db) shouldBe empty
+  }
+
+  test("shuffle service should be robust to corrupt registered executor file") {
+    s1 = new YarnShuffleService
+    s1.init(yarnConfig)
+    val app1Id = ApplicationId.newInstance(0, 1)
+    val app1Data = makeAppInfo("user", app1Id)
+    s1.initializeApplication(app1Data)
+
+    val execStateFile = s1.registeredExecutorFile
+    val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER)
+
+    val blockHandler = s1.blockHandler
+    val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler)
+    ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile)
+
+    blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1)
+
+    // now we pretend the shuffle service goes down, and comes back up.  But we'll also
+    // make a corrupt registeredExecutor File
+    s1.stop()
+
+    execStateFile.listFiles().foreach{_.delete()}
+
+    val out = new DataOutputStream(new FileOutputStream(execStateFile + "/CURRENT"))
+    out.writeInt(42)
+    out.close()
+
+    s2 = new YarnShuffleService
+    s2.init(yarnConfig)
+    s2.registeredExecutorFile should be (execStateFile)
+
+    val handler2 = s2.blockHandler
+    val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2)
+
+    // we re-initialize app1, but since the file was corrupt there is nothing we can do about it ...
+    s2.initializeApplication(app1Data)
+    // however, when we initialize a totally new app2, everything is still happy
+    val app2Id = ApplicationId.newInstance(0, 2)
+    val app2Data = makeAppInfo("user", app2Id)
+    s2.initializeApplication(app2Data)
+    val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER)
+    resolver2.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2)
+    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (Some(shuffleInfo2))
+    s2.stop()
+
+    // another stop & restart should be fine though (eg., we recover from previous corruption)
+    s3 = new YarnShuffleService
+    s3.init(yarnConfig)
+    s3.registeredExecutorFile should be (execStateFile)
+    val handler3 = s3.blockHandler
+    val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3)
+
+    s3.initializeApplication(app2Data)
+    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (Some(shuffleInfo2))
+    s3.stop()
+  }
+
+  test("get correct recovery path") {
+    // Test recovery path is set outside the shuffle service, this is to simulate NM recovery
+    // enabled scenario, where recovery path will be set by yarn.
+    s1 = new YarnShuffleService
+    val recoveryPath = new Path(Utils.createTempDir().toURI)
+    s1.setRecoveryPath(recoveryPath)
+
+    s1.init(yarnConfig)
+    s1._recoveryPath should be (recoveryPath)
+    s1.stop()
+
+    // Test recovery path is set inside the shuffle service, this will be happened when NM
+    // recovery is not enabled or there's no NM recovery (Hadoop 2.5-).
+    s2 = new YarnShuffleService
+    s2.init(yarnConfig)
+    s2._recoveryPath should be
+      (new Path(yarnConfig.getTrimmedStrings("yarn.nodemanager.local-dirs")(0)))
+    s2.stop()
+  }
+
+  test("moving recovery file from NM local dir to recovery path") {
+    // This is to test when Hadoop is upgrade to 2.5+ and NM recovery is enabled, we should move
+    // old recovery file to the new path to keep compatibility
+
+    // Simulate s1 is running on old version of Hadoop in which recovery file is in the NM local
+    // dir.
+    s1 = new YarnShuffleService
+    // set auth to true to test the secrets recovery
+    yarnConfig.setBoolean(SecurityManager.SPARK_AUTH_CONF, true)
+    s1.init(yarnConfig)
+    val app1Id = ApplicationId.newInstance(0, 1)
+    val app1Data = makeAppInfo("user", app1Id)
+    s1.initializeApplication(app1Data)
+    val app2Id = ApplicationId.newInstance(0, 2)
+    val app2Data = makeAppInfo("user", app2Id)
+    s1.initializeApplication(app2Data)
+
+    assert(s1.secretManager.getSecretKey(app1Id.toString()) != null)
+    assert(s1.secretManager.getSecretKey(app2Id.toString()) != null)
+
+    val execStateFile = s1.registeredExecutorFile
+    execStateFile should not be (null)
+    val secretsFile = s1.secretsFile
+    secretsFile should not be (null)
+    val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER)
+    val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER)
+
+    val blockHandler = s1.blockHandler
+    val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler)
+    ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile)
+
+    blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1)
+    blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2)
+    ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", blockResolver) should
+      be (Some(shuffleInfo1))
+    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", blockResolver) should
+      be (Some(shuffleInfo2))
+
+    assert(execStateFile.exists(), s"$execStateFile did not exist")
+
+    s1.stop()
+
+    // Simulate s2 is running on Hadoop 2.5+ with NM recovery is enabled.
+    assert(execStateFile.exists())
+    val recoveryPath = new Path(Utils.createTempDir().toURI)
+    s2 = new YarnShuffleService
+    s2.setRecoveryPath(recoveryPath)
+    s2.init(yarnConfig)
+
+    // Ensure that s2 has loaded known apps from the secrets db.
+    assert(s2.secretManager.getSecretKey(app1Id.toString()) != null)
+    assert(s2.secretManager.getSecretKey(app2Id.toString()) != null)
+
+    val execStateFile2 = s2.registeredExecutorFile
+    val secretsFile2 = s2.secretsFile
+
+    recoveryPath.toString should be (new Path(execStateFile2.getParentFile.toURI).toString)
+    recoveryPath.toString should be (new Path(secretsFile2.getParentFile.toURI).toString)
+    eventually(timeout(10 seconds), interval(5 millis)) {
+      assert(!execStateFile.exists())
+    }
+    eventually(timeout(10 seconds), interval(5 millis)) {
+      assert(!secretsFile.exists())
+    }
+
+    val handler2 = s2.blockHandler
+    val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2)
+
+    // now we reinitialize only one of the apps, and expect yarn to tell us that app2 was stopped
+    // during the restart
+    // Since recovery file is got from old path, so the previous state should be stored.
+    s2.initializeApplication(app1Data)
+    s2.stopApplication(new ApplicationTerminationContext(app2Id))
+    ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (Some(shuffleInfo1))
+    ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None)
+
+    s2.stop()
+  }
+
+  test("service throws error if cannot start") {
+    // Set up a read-only local dir.
+    val roDir = Utils.createTempDir()
+    Files.setPosixFilePermissions(roDir.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE))
+    yarnConfig.set(YarnConfiguration.NM_LOCAL_DIRS, roDir.getAbsolutePath())
+
+    // Try to start the shuffle service, it should fail.
+    val service = new YarnShuffleService()
+
+    try {
+      val error = intercept[ServiceStateException] {
+        service.init(yarnConfig)
+      }
+      assert(error.getCause().isInstanceOf[IOException])
+    } finally {
+      service.stop()
+      Files.setPosixFilePermissions(roDir.toPath(),
+        EnumSet.of(OWNER_READ, OWNER_WRITE, OWNER_EXECUTE))
+    }
+  }
+
+  private def makeAppInfo(user: String, appId: ApplicationId): ApplicationInitializationContext = {
+    val secret = ByteBuffer.wrap(new Array[Byte](0))
+    new ApplicationInitializationContext(user, appId, secret)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala
new file mode 100644
index 0000000..db322cd
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.network.yarn
+
+import java.io.File
+
+/**
+ * just a cheat to get package-visible members in tests
+ */
+object YarnTestAccessor {
+  def getShuffleServicePort: Int = {
+    YarnShuffleService.boundPort
+  }
+
+  def getShuffleServiceInstance: YarnShuffleService = {
+    YarnShuffleService.instance
+  }
+
+  def getRegisteredExecutorFile(service: YarnShuffleService): File = {
+    service.registeredExecutorFile
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala
new file mode 100644
index 0000000..6ea7984
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler.cluster
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite}
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+
+/**
+ * Test the integration with [[SchedulerExtensionServices]]
+ */
+class ExtensionServiceIntegrationSuite extends SparkFunSuite
+  with LocalSparkContext with BeforeAndAfter
+  with Logging {
+
+  val applicationId = new StubApplicationId(0, 1111L)
+  val attemptId = new StubApplicationAttemptId(applicationId, 1)
+
+  /*
+   * Setup phase creates the spark context
+   */
+  before {
+    val sparkConf = new SparkConf()
+    sparkConf.set(SCHEDULER_SERVICES, Seq(classOf[SimpleExtensionService].getName()))
+    sparkConf.setMaster("local").setAppName("ExtensionServiceIntegrationSuite")
+    sc = new SparkContext(sparkConf)
+  }
+
+  test("Instantiate") {
+    val services = new SchedulerExtensionServices()
+    assertResult(Nil, "non-nil service list") {
+      services.getServices
+    }
+    services.start(SchedulerExtensionServiceBinding(sc, applicationId))
+    services.stop()
+  }
+
+  test("Contains SimpleExtensionService Service") {
+    val services = new SchedulerExtensionServices()
+    try {
+      services.start(SchedulerExtensionServiceBinding(sc, applicationId))
+      val serviceList = services.getServices
+      assert(serviceList.nonEmpty, "empty service list")
+      val (service :: Nil) = serviceList
+      val simpleService = service.asInstanceOf[SimpleExtensionService]
+      assert(simpleService.started.get, "service not started")
+      services.stop()
+      assert(!simpleService.started.get, "service not stopped")
+    } finally {
+      services.stop()
+    }
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala
new file mode 100644
index 0000000..9b8c98c
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala
@@ -0,0 +1,34 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicBoolean
+
+private[spark] class SimpleExtensionService extends SchedulerExtensionService {
+
+  /** started flag; set in the `start()` call, stopped in `stop()`. */
+  val started = new AtomicBoolean(false)
+
+  override def start(binding: SchedulerExtensionServiceBinding): Unit = {
+    started.set(true)
+  }
+
+  override def stop(): Unit = {
+    started.set(false)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala
new file mode 100644
index 0000000..4b57b95
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler.cluster
+
+import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
+
+/**
+ * A stub application ID; can be set in constructor and/or updated later.
+ * @param applicationId application ID
+ * @param attempt an attempt counter
+ */
+class StubApplicationAttemptId(var applicationId: ApplicationId, var attempt: Int)
+    extends ApplicationAttemptId {
+
+  override def setApplicationId(appID: ApplicationId): Unit = {
+    applicationId = appID
+  }
+
+  override def getAttemptId: Int = {
+    attempt
+  }
+
+  override def setAttemptId(attemptId: Int): Unit = {
+    attempt = attemptId
+  }
+
+  override def getApplicationId: ApplicationId = {
+    applicationId
+  }
+
+  override def build(): Unit = {
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala
new file mode 100644
index 0000000..bffa0e0
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler.cluster
+
+import org.apache.hadoop.yarn.api.records.ApplicationId
+
+/**
+ * Simple Testing Application Id; ID and cluster timestamp are set in constructor
+ * and cannot be updated.
+ * @param id app id
+ * @param clusterTimestamp timestamp
+ */
+private[spark] class StubApplicationId(id: Int, clusterTimestamp: Long) extends ApplicationId {
+  override def getId: Int = {
+    id
+  }
+
+  override def getClusterTimestamp: Long = {
+    clusterTimestamp
+  }
+
+  override def setId(id: Int): Unit = {}
+
+  override def setClusterTimestamp(clusterTimestamp: Long): Unit = {}
+
+  override def build(): Unit = {}
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/yarn/pom.xml b/yarn/pom.xml
deleted file mode 100644
index 91e3437..0000000
--- a/yarn/pom.xml
+++ /dev/null
@@ -1,215 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one or more
-  ~ contributor license agreements.  See the NOTICE file distributed with
-  ~ this work for additional information regarding copyright ownership.
-  ~ The ASF licenses this file to You under the Apache License, Version 2.0
-  ~ (the "License"); you may not use this file except in compliance with
-  ~ the License.  You may obtain a copy of the License at
-  ~
-  ~    http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS,
-  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~ See the License for the specific language governing permissions and
-  ~ limitations under the License.
-  -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.spark</groupId>
-    <artifactId>spark-parent_2.11</artifactId>
-    <version>2.2.0-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
-
-  <artifactId>spark-yarn_2.11</artifactId>
-  <packaging>jar</packaging>
-  <name>Spark Project YARN</name>
-  <properties>
-    <sbt.project.name>yarn</sbt.project.name>
-    <jersey-1.version>1.9</jersey-1.version>
-  </properties>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-network-yarn_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-tags_${scala.binary.version}</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-server-web-proxy</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-client</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client</artifactId>
-    </dependency>
-
-    <!-- Explicit listing of transitive deps that are shaded. Otherwise, odd compiler crashes. -->
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-server</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-plus</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-http</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-servlet</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-servlets</artifactId>
-    </dependency>
-    <!-- End of shaded deps. -->
-
-    <!--
-      SPARK-10059: Explicitly add JSP dependencies for tests since the MiniYARN cluster needs them.
-    -->
-    <dependency>
-      <groupId>org.eclipse.jetty.orbit</groupId>
-      <artifactId>javax.servlet.jsp</artifactId>
-      <version>2.2.0.v201112011158</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty.orbit</groupId>
-      <artifactId>javax.servlet.jsp.jstl</artifactId>
-      <version>1.2.0.v201105211821</version>
-      <scope>test</scope>
-    </dependency>
-
-     <!--
-    See SPARK-3710. hadoop-yarn-server-tests in Hadoop 2.2 fails to pull some needed
-    dependencies, so they need to be added manually for the tests to work.
-    -->
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-server-tests</artifactId>
-      <classifier>tests</classifier>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-core</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
-      <version>6.1.26</version>
-      <exclusions>
-       <exclusion>
-        <groupId>org.mortbay.jetty</groupId>
-         <artifactId>servlet-api</artifactId>
-       </exclusion>
-      </exclusions>
-      <scope>test</scope>
-     </dependency>
-
-     <!--
-       Jersey 1 dependencies only required for YARN integration testing. Creating a YARN cluster
-       in the JVM requires starting a Jersey 1-based web application.
-     -->
-     <dependency>
-       <groupId>com.sun.jersey</groupId>
-       <artifactId>jersey-core</artifactId>
-       <scope>test</scope>
-       <version>${jersey-1.version}</version>
-     </dependency>
-     <dependency>
-       <groupId>com.sun.jersey</groupId>
-       <artifactId>jersey-json</artifactId>
-       <scope>test</scope>
-       <version>${jersey-1.version}</version>
-     </dependency>
-     <dependency>
-       <groupId>com.sun.jersey</groupId>
-       <artifactId>jersey-server</artifactId>
-       <scope>test</scope>
-       <version>${jersey-1.version}</version>
-     </dependency>
-     <dependency>
-       <groupId>com.sun.jersey.contribs</groupId>
-       <artifactId>jersey-guice</artifactId>
-       <scope>test</scope>
-       <version>${jersey-1.version}</version>
-     </dependency>
-
-    <!--
-     Testing Hive reflection needs hive on the test classpath only.
-     It doesn't need the spark hive modules, so the -Phive flag is not checked.
-      -->
-    <dependency>
-      <groupId>${hive.group}</groupId>
-      <artifactId>hive-exec</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>${hive.group}</groupId>
-      <artifactId>hive-metastore</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.thrift</groupId>
-      <artifactId>libthrift</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.thrift</groupId>
-      <artifactId>libfb303</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
-  </build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
----------------------------------------------------------------------
diff --git a/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider b/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
deleted file mode 100644
index 22ead56..0000000
--- a/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
+++ /dev/null
@@ -1,3 +0,0 @@
-org.apache.spark.deploy.yarn.security.HDFSCredentialProvider
-org.apache.spark.deploy.yarn.security.HBaseCredentialProvider
-org.apache.spark.deploy.yarn.security.HiveCredentialProvider

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
----------------------------------------------------------------------
diff --git a/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
deleted file mode 100644
index 6e8a1eb..0000000
--- a/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
+++ /dev/null
@@ -1 +0,0 @@
-org.apache.spark.scheduler.cluster.YarnClusterManager


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


[05/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
deleted file mode 100644
index be419ce..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ /dev/null
@@ -1,1541 +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.deploy.yarn
-
-import java.io.{File, FileOutputStream, IOException, OutputStreamWriter}
-import java.net.{InetAddress, UnknownHostException, URI}
-import java.nio.ByteBuffer
-import java.nio.charset.StandardCharsets
-import java.util.{Properties, UUID}
-import java.util.zip.{ZipEntry, ZipOutputStream}
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map}
-import scala.util.{Failure, Success, Try}
-import scala.util.control.NonFatal
-
-import com.google.common.base.Objects
-import com.google.common.io.Files
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs._
-import org.apache.hadoop.fs.permission.FsPermission
-import org.apache.hadoop.io.DataOutputBuffer
-import org.apache.hadoop.mapreduce.MRJobConfig
-import org.apache.hadoop.security.{Credentials, UserGroupInformation}
-import org.apache.hadoop.util.StringUtils
-import org.apache.hadoop.yarn.api._
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-import org.apache.hadoop.yarn.api.protocolrecords._
-import org.apache.hadoop.yarn.api.records._
-import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication}
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException
-import org.apache.hadoop.yarn.util.Records
-
-import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkException}
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager
-import org.apache.spark.internal.Logging
-import org.apache.spark.internal.config._
-import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils}
-import org.apache.spark.util.{CallerContext, Utils}
-
-private[spark] class Client(
-    val args: ClientArguments,
-    val hadoopConf: Configuration,
-    val sparkConf: SparkConf)
-  extends Logging {
-
-  import Client._
-  import YarnSparkHadoopUtil._
-
-  def this(clientArgs: ClientArguments, spConf: SparkConf) =
-    this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf)
-
-  private val yarnClient = YarnClient.createYarnClient
-  private val yarnConf = new YarnConfiguration(hadoopConf)
-
-  private val isClusterMode = sparkConf.get("spark.submit.deployMode", "client") == "cluster"
-
-  // AM related configurations
-  private val amMemory = if (isClusterMode) {
-    sparkConf.get(DRIVER_MEMORY).toInt
-  } else {
-    sparkConf.get(AM_MEMORY).toInt
-  }
-  private val amMemoryOverhead = {
-    val amMemoryOverheadEntry = if (isClusterMode) DRIVER_MEMORY_OVERHEAD else AM_MEMORY_OVERHEAD
-    sparkConf.get(amMemoryOverheadEntry).getOrElse(
-      math.max((MEMORY_OVERHEAD_FACTOR * amMemory).toLong, MEMORY_OVERHEAD_MIN)).toInt
-  }
-  private val amCores = if (isClusterMode) {
-    sparkConf.get(DRIVER_CORES)
-  } else {
-    sparkConf.get(AM_CORES)
-  }
-
-  // Executor related configurations
-  private val executorMemory = sparkConf.get(EXECUTOR_MEMORY)
-  private val executorMemoryOverhead = sparkConf.get(EXECUTOR_MEMORY_OVERHEAD).getOrElse(
-    math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toLong, MEMORY_OVERHEAD_MIN)).toInt
-
-  private val distCacheMgr = new ClientDistributedCacheManager()
-
-  private var loginFromKeytab = false
-  private var principal: String = null
-  private var keytab: String = null
-  private var credentials: Credentials = null
-
-  private val launcherBackend = new LauncherBackend() {
-    override def onStopRequest(): Unit = {
-      if (isClusterMode && appId != null) {
-        yarnClient.killApplication(appId)
-      } else {
-        setState(SparkAppHandle.State.KILLED)
-        stop()
-      }
-    }
-  }
-  private val fireAndForget = isClusterMode && !sparkConf.get(WAIT_FOR_APP_COMPLETION)
-
-  private var appId: ApplicationId = null
-
-  // The app staging dir based on the STAGING_DIR configuration if configured
-  // otherwise based on the users home directory.
-  private val appStagingBaseDir = sparkConf.get(STAGING_DIR).map { new Path(_) }
-    .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory())
-
-  private val credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
-
-  def reportLauncherState(state: SparkAppHandle.State): Unit = {
-    launcherBackend.setState(state)
-  }
-
-  def stop(): Unit = {
-    launcherBackend.close()
-    yarnClient.stop()
-    // Unset YARN mode system env variable, to allow switching between cluster types.
-    System.clearProperty("SPARK_YARN_MODE")
-  }
-
-  /**
-   * Submit an application running our ApplicationMaster to the ResourceManager.
-   *
-   * The stable Yarn API provides a convenience method (YarnClient#createApplication) for
-   * creating applications and setting up the application submission context. This was not
-   * available in the alpha API.
-   */
-  def submitApplication(): ApplicationId = {
-    var appId: ApplicationId = null
-    try {
-      launcherBackend.connect()
-      // Setup the credentials before doing anything else,
-      // so we have don't have issues at any point.
-      setupCredentials()
-      yarnClient.init(yarnConf)
-      yarnClient.start()
-
-      logInfo("Requesting a new application from cluster with %d NodeManagers"
-        .format(yarnClient.getYarnClusterMetrics.getNumNodeManagers))
-
-      // Get a new application from our RM
-      val newApp = yarnClient.createApplication()
-      val newAppResponse = newApp.getNewApplicationResponse()
-      appId = newAppResponse.getApplicationId()
-      reportLauncherState(SparkAppHandle.State.SUBMITTED)
-      launcherBackend.setAppId(appId.toString)
-
-      new CallerContext("CLIENT", sparkConf.get(APP_CALLER_CONTEXT),
-        Option(appId.toString)).setCurrentContext()
-
-      // Verify whether the cluster has enough resources for our AM
-      verifyClusterResources(newAppResponse)
-
-      // Set up the appropriate contexts to launch our AM
-      val containerContext = createContainerLaunchContext(newAppResponse)
-      val appContext = createApplicationSubmissionContext(newApp, containerContext)
-
-      // Finally, submit and monitor the application
-      logInfo(s"Submitting application $appId to ResourceManager")
-      yarnClient.submitApplication(appContext)
-      appId
-    } catch {
-      case e: Throwable =>
-        if (appId != null) {
-          cleanupStagingDir(appId)
-        }
-        throw e
-    }
-  }
-
-  /**
-   * Cleanup application staging directory.
-   */
-  private def cleanupStagingDir(appId: ApplicationId): Unit = {
-    val stagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId))
-    try {
-      val preserveFiles = sparkConf.get(PRESERVE_STAGING_FILES)
-      val fs = stagingDirPath.getFileSystem(hadoopConf)
-      if (!preserveFiles && fs.delete(stagingDirPath, true)) {
-        logInfo(s"Deleted staging directory $stagingDirPath")
-      }
-    } catch {
-      case ioe: IOException =>
-        logWarning("Failed to cleanup staging dir " + stagingDirPath, ioe)
-    }
-  }
-
-  /**
-   * Set up the context for submitting our ApplicationMaster.
-   * This uses the YarnClientApplication not available in the Yarn alpha API.
-   */
-  def createApplicationSubmissionContext(
-      newApp: YarnClientApplication,
-      containerContext: ContainerLaunchContext): ApplicationSubmissionContext = {
-    val appContext = newApp.getApplicationSubmissionContext
-    appContext.setApplicationName(sparkConf.get("spark.app.name", "Spark"))
-    appContext.setQueue(sparkConf.get(QUEUE_NAME))
-    appContext.setAMContainerSpec(containerContext)
-    appContext.setApplicationType("SPARK")
-
-    sparkConf.get(APPLICATION_TAGS).foreach { tags =>
-      try {
-        // The setApplicationTags method was only introduced in Hadoop 2.4+, so we need to use
-        // reflection to set it, printing a warning if a tag was specified but the YARN version
-        // doesn't support it.
-        val method = appContext.getClass().getMethod(
-          "setApplicationTags", classOf[java.util.Set[String]])
-        method.invoke(appContext, new java.util.HashSet[String](tags.asJava))
-      } catch {
-        case e: NoSuchMethodException =>
-          logWarning(s"Ignoring ${APPLICATION_TAGS.key} because this version of " +
-            "YARN does not support it")
-      }
-    }
-    sparkConf.get(MAX_APP_ATTEMPTS) match {
-      case Some(v) => appContext.setMaxAppAttempts(v)
-      case None => logDebug(s"${MAX_APP_ATTEMPTS.key} is not set. " +
-          "Cluster's default value will be used.")
-    }
-
-    sparkConf.get(AM_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS).foreach { interval =>
-      try {
-        val method = appContext.getClass().getMethod(
-          "setAttemptFailuresValidityInterval", classOf[Long])
-        method.invoke(appContext, interval: java.lang.Long)
-      } catch {
-        case e: NoSuchMethodException =>
-          logWarning(s"Ignoring ${AM_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS.key} because " +
-            "the version of YARN does not support it")
-      }
-    }
-
-    val capability = Records.newRecord(classOf[Resource])
-    capability.setMemory(amMemory + amMemoryOverhead)
-    capability.setVirtualCores(amCores)
-
-    sparkConf.get(AM_NODE_LABEL_EXPRESSION) match {
-      case Some(expr) =>
-        try {
-          val amRequest = Records.newRecord(classOf[ResourceRequest])
-          amRequest.setResourceName(ResourceRequest.ANY)
-          amRequest.setPriority(Priority.newInstance(0))
-          amRequest.setCapability(capability)
-          amRequest.setNumContainers(1)
-          val method = amRequest.getClass.getMethod("setNodeLabelExpression", classOf[String])
-          method.invoke(amRequest, expr)
-
-          val setResourceRequestMethod =
-            appContext.getClass.getMethod("setAMContainerResourceRequest", classOf[ResourceRequest])
-          setResourceRequestMethod.invoke(appContext, amRequest)
-        } catch {
-          case e: NoSuchMethodException =>
-            logWarning(s"Ignoring ${AM_NODE_LABEL_EXPRESSION.key} because the version " +
-              "of YARN does not support it")
-            appContext.setResource(capability)
-        }
-      case None =>
-        appContext.setResource(capability)
-    }
-
-    sparkConf.get(ROLLED_LOG_INCLUDE_PATTERN).foreach { includePattern =>
-      try {
-        val logAggregationContext = Records.newRecord(
-          Utils.classForName("org.apache.hadoop.yarn.api.records.LogAggregationContext"))
-          .asInstanceOf[Object]
-
-        val setRolledLogsIncludePatternMethod =
-          logAggregationContext.getClass.getMethod("setRolledLogsIncludePattern", classOf[String])
-        setRolledLogsIncludePatternMethod.invoke(logAggregationContext, includePattern)
-
-        sparkConf.get(ROLLED_LOG_EXCLUDE_PATTERN).foreach { excludePattern =>
-          val setRolledLogsExcludePatternMethod =
-            logAggregationContext.getClass.getMethod("setRolledLogsExcludePattern", classOf[String])
-          setRolledLogsExcludePatternMethod.invoke(logAggregationContext, excludePattern)
-        }
-
-        val setLogAggregationContextMethod =
-          appContext.getClass.getMethod("setLogAggregationContext",
-            Utils.classForName("org.apache.hadoop.yarn.api.records.LogAggregationContext"))
-        setLogAggregationContextMethod.invoke(appContext, logAggregationContext)
-      } catch {
-        case NonFatal(e) =>
-          logWarning(s"Ignoring ${ROLLED_LOG_INCLUDE_PATTERN.key} because the version of YARN " +
-            s"does not support it", e)
-      }
-    }
-
-    appContext
-  }
-
-  /** Set up security tokens for launching our ApplicationMaster container. */
-  private def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = {
-    val dob = new DataOutputBuffer
-    credentials.writeTokenStorageToStream(dob)
-    amContainer.setTokens(ByteBuffer.wrap(dob.getData))
-  }
-
-  /** Get the application report from the ResourceManager for an application we have submitted. */
-  def getApplicationReport(appId: ApplicationId): ApplicationReport =
-    yarnClient.getApplicationReport(appId)
-
-  /**
-   * Return the security token used by this client to communicate with the ApplicationMaster.
-   * If no security is enabled, the token returned by the report is null.
-   */
-  private def getClientToken(report: ApplicationReport): String =
-    Option(report.getClientToAMToken).map(_.toString).getOrElse("")
-
-  /**
-   * Fail fast if we have requested more resources per container than is available in the cluster.
-   */
-  private def verifyClusterResources(newAppResponse: GetNewApplicationResponse): Unit = {
-    val maxMem = newAppResponse.getMaximumResourceCapability().getMemory()
-    logInfo("Verifying our application has not requested more than the maximum " +
-      s"memory capability of the cluster ($maxMem MB per container)")
-    val executorMem = executorMemory + executorMemoryOverhead
-    if (executorMem > maxMem) {
-      throw new IllegalArgumentException(s"Required executor memory ($executorMemory" +
-        s"+$executorMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster! " +
-        "Please check the values of 'yarn.scheduler.maximum-allocation-mb' and/or " +
-        "'yarn.nodemanager.resource.memory-mb'.")
-    }
-    val amMem = amMemory + amMemoryOverhead
-    if (amMem > maxMem) {
-      throw new IllegalArgumentException(s"Required AM memory ($amMemory" +
-        s"+$amMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster! " +
-        "Please increase the value of 'yarn.scheduler.maximum-allocation-mb'.")
-    }
-    logInfo("Will allocate AM container, with %d MB memory including %d MB overhead".format(
-      amMem,
-      amMemoryOverhead))
-
-    // We could add checks to make sure the entire cluster has enough resources but that involves
-    // getting all the node reports and computing ourselves.
-  }
-
-  /**
-   * Copy the given file to a remote file system (e.g. HDFS) if needed.
-   * The file is only copied if the source and destination file systems are different. This is used
-   * for preparing resources for launching the ApplicationMaster container. Exposed for testing.
-   */
-  private[yarn] def copyFileToRemote(
-      destDir: Path,
-      srcPath: Path,
-      replication: Short,
-      force: Boolean = false,
-      destName: Option[String] = None): Path = {
-    val destFs = destDir.getFileSystem(hadoopConf)
-    val srcFs = srcPath.getFileSystem(hadoopConf)
-    var destPath = srcPath
-    if (force || !compareFs(srcFs, destFs)) {
-      destPath = new Path(destDir, destName.getOrElse(srcPath.getName()))
-      logInfo(s"Uploading resource $srcPath -> $destPath")
-      FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf)
-      destFs.setReplication(destPath, replication)
-      destFs.setPermission(destPath, new FsPermission(APP_FILE_PERMISSION))
-    } else {
-      logInfo(s"Source and destination file systems are the same. Not copying $srcPath")
-    }
-    // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
-    // version shows the specific version in the distributed cache configuration
-    val qualifiedDestPath = destFs.makeQualified(destPath)
-    val fc = FileContext.getFileContext(qualifiedDestPath.toUri(), hadoopConf)
-    fc.resolvePath(qualifiedDestPath)
-  }
-
-  /**
-   * Upload any resources to the distributed cache if needed. If a resource is intended to be
-   * consumed locally, set up the appropriate config for downstream code to handle it properly.
-   * This is used for setting up a container launch context for our ApplicationMaster.
-   * Exposed for testing.
-   */
-  def prepareLocalResources(
-      destDir: Path,
-      pySparkArchives: Seq[String]): HashMap[String, LocalResource] = {
-    logInfo("Preparing resources for our AM container")
-    // Upload Spark and the application JAR to the remote file system if necessary,
-    // and add them as local resources to the application master.
-    val fs = destDir.getFileSystem(hadoopConf)
-
-    // Merge credentials obtained from registered providers
-    val nearestTimeOfNextRenewal = credentialManager.obtainCredentials(hadoopConf, credentials)
-
-    if (credentials != null) {
-      logDebug(YarnSparkHadoopUtil.get.dumpTokens(credentials).mkString("\n"))
-    }
-
-    // If we use principal and keytab to login, also credentials can be renewed some time
-    // after current time, we should pass the next renewal and updating time to credential
-    // renewer and updater.
-    if (loginFromKeytab && nearestTimeOfNextRenewal > System.currentTimeMillis() &&
-      nearestTimeOfNextRenewal != Long.MaxValue) {
-
-      // Valid renewal time is 75% of next renewal time, and the valid update time will be
-      // slightly later then renewal time (80% of next renewal time). This is to make sure
-      // credentials are renewed and updated before expired.
-      val currTime = System.currentTimeMillis()
-      val renewalTime = (nearestTimeOfNextRenewal - currTime) * 0.75 + currTime
-      val updateTime = (nearestTimeOfNextRenewal - currTime) * 0.8 + currTime
-
-      sparkConf.set(CREDENTIALS_RENEWAL_TIME, renewalTime.toLong)
-      sparkConf.set(CREDENTIALS_UPDATE_TIME, updateTime.toLong)
-    }
-
-    // Used to keep track of URIs added to the distributed cache. If the same URI is added
-    // multiple times, YARN will fail to launch containers for the app with an internal
-    // error.
-    val distributedUris = new HashSet[String]
-    // Used to keep track of URIs(files) added to the distribute cache have the same name. If
-    // same name but different path files are added multiple time, YARN will fail to launch
-    // containers for the app with an internal error.
-    val distributedNames = new HashSet[String]
-
-    val replication = sparkConf.get(STAGING_FILE_REPLICATION).map(_.toShort)
-      .getOrElse(fs.getDefaultReplication(destDir))
-    val localResources = HashMap[String, LocalResource]()
-    FileSystem.mkdirs(fs, destDir, new FsPermission(STAGING_DIR_PERMISSION))
-
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-
-    def addDistributedUri(uri: URI): Boolean = {
-      val uriStr = uri.toString()
-      val fileName = new File(uri.getPath).getName
-      if (distributedUris.contains(uriStr)) {
-        logWarning(s"Same path resource $uri added multiple times to distributed cache.")
-        false
-      } else if (distributedNames.contains(fileName)) {
-        logWarning(s"Same name resource $uri added multiple times to distributed cache")
-        false
-      } else {
-        distributedUris += uriStr
-        distributedNames += fileName
-        true
-      }
-    }
-
-    /**
-     * Distribute a file to the cluster.
-     *
-     * If the file's path is a "local:" URI, it's actually not distributed. Other files are copied
-     * to HDFS (if not already there) and added to the application's distributed cache.
-     *
-     * @param path URI of the file to distribute.
-     * @param resType Type of resource being distributed.
-     * @param destName Name of the file in the distributed cache.
-     * @param targetDir Subdirectory where to place the file.
-     * @param appMasterOnly Whether to distribute only to the AM.
-     * @return A 2-tuple. First item is whether the file is a "local:" URI. Second item is the
-     *         localized path for non-local paths, or the input `path` for local paths.
-     *         The localized path will be null if the URI has already been added to the cache.
-     */
-    def distribute(
-        path: String,
-        resType: LocalResourceType = LocalResourceType.FILE,
-        destName: Option[String] = None,
-        targetDir: Option[String] = None,
-        appMasterOnly: Boolean = false): (Boolean, String) = {
-      val trimmedPath = path.trim()
-      val localURI = Utils.resolveURI(trimmedPath)
-      if (localURI.getScheme != LOCAL_SCHEME) {
-        if (addDistributedUri(localURI)) {
-          val localPath = getQualifiedLocalPath(localURI, hadoopConf)
-          val linkname = targetDir.map(_ + "/").getOrElse("") +
-            destName.orElse(Option(localURI.getFragment())).getOrElse(localPath.getName())
-          val destPath = copyFileToRemote(destDir, localPath, replication)
-          val destFs = FileSystem.get(destPath.toUri(), hadoopConf)
-          distCacheMgr.addResource(
-            destFs, hadoopConf, destPath, localResources, resType, linkname, statCache,
-            appMasterOnly = appMasterOnly)
-          (false, linkname)
-        } else {
-          (false, null)
-        }
-      } else {
-        (true, trimmedPath)
-      }
-    }
-
-    // If we passed in a keytab, make sure we copy the keytab to the staging directory on
-    // HDFS, and setup the relevant environment vars, so the AM can login again.
-    if (loginFromKeytab) {
-      logInfo("To enable the AM to login from keytab, credentials are being copied over to the AM" +
-        " via the YARN Secure Distributed Cache.")
-      val (_, localizedPath) = distribute(keytab,
-        destName = sparkConf.get(KEYTAB),
-        appMasterOnly = true)
-      require(localizedPath != null, "Keytab file already distributed.")
-    }
-
-    /**
-     * Add Spark to the cache. There are two settings that control what files to add to the cache:
-     * - if a Spark archive is defined, use the archive. The archive is expected to contain
-     *   jar files at its root directory.
-     * - if a list of jars is provided, filter the non-local ones, resolve globs, and
-     *   add the found files to the cache.
-     *
-     * Note that the archive cannot be a "local" URI. If none of the above settings are found,
-     * then upload all files found in $SPARK_HOME/jars.
-     */
-    val sparkArchive = sparkConf.get(SPARK_ARCHIVE)
-    if (sparkArchive.isDefined) {
-      val archive = sparkArchive.get
-      require(!isLocalUri(archive), s"${SPARK_ARCHIVE.key} cannot be a local URI.")
-      distribute(Utils.resolveURI(archive).toString,
-        resType = LocalResourceType.ARCHIVE,
-        destName = Some(LOCALIZED_LIB_DIR))
-    } else {
-      sparkConf.get(SPARK_JARS) match {
-        case Some(jars) =>
-          // Break the list of jars to upload, and resolve globs.
-          val localJars = new ArrayBuffer[String]()
-          jars.foreach { jar =>
-            if (!isLocalUri(jar)) {
-              val path = getQualifiedLocalPath(Utils.resolveURI(jar), hadoopConf)
-              val pathFs = FileSystem.get(path.toUri(), hadoopConf)
-              pathFs.globStatus(path).filter(_.isFile()).foreach { entry =>
-                distribute(entry.getPath().toUri().toString(),
-                  targetDir = Some(LOCALIZED_LIB_DIR))
-              }
-            } else {
-              localJars += jar
-            }
-          }
-
-          // Propagate the local URIs to the containers using the configuration.
-          sparkConf.set(SPARK_JARS, localJars)
-
-        case None =>
-          // No configuration, so fall back to uploading local jar files.
-          logWarning(s"Neither ${SPARK_JARS.key} nor ${SPARK_ARCHIVE.key} is set, falling back " +
-            "to uploading libraries under SPARK_HOME.")
-          val jarsDir = new File(YarnCommandBuilderUtils.findJarsDir(
-            sparkConf.getenv("SPARK_HOME")))
-          val jarsArchive = File.createTempFile(LOCALIZED_LIB_DIR, ".zip",
-            new File(Utils.getLocalDir(sparkConf)))
-          val jarsStream = new ZipOutputStream(new FileOutputStream(jarsArchive))
-
-          try {
-            jarsStream.setLevel(0)
-            jarsDir.listFiles().foreach { f =>
-              if (f.isFile && f.getName.toLowerCase().endsWith(".jar") && f.canRead) {
-                jarsStream.putNextEntry(new ZipEntry(f.getName))
-                Files.copy(f, jarsStream)
-                jarsStream.closeEntry()
-              }
-            }
-          } finally {
-            jarsStream.close()
-          }
-
-          distribute(jarsArchive.toURI.getPath,
-            resType = LocalResourceType.ARCHIVE,
-            destName = Some(LOCALIZED_LIB_DIR))
-      }
-    }
-
-    /**
-     * Copy user jar to the distributed cache if their scheme is not "local".
-     * Otherwise, set the corresponding key in our SparkConf to handle it downstream.
-     */
-    Option(args.userJar).filter(_.trim.nonEmpty).foreach { jar =>
-      val (isLocal, localizedPath) = distribute(jar, destName = Some(APP_JAR_NAME))
-      if (isLocal) {
-        require(localizedPath != null, s"Path $jar already distributed")
-        // If the resource is intended for local use only, handle this downstream
-        // by setting the appropriate property
-        sparkConf.set(APP_JAR, localizedPath)
-      }
-    }
-
-    /**
-     * Do the same for any additional resources passed in through ClientArguments.
-     * Each resource category is represented by a 3-tuple of:
-     *   (1) comma separated list of resources in this category,
-     *   (2) resource type, and
-     *   (3) whether to add these resources to the classpath
-     */
-    val cachedSecondaryJarLinks = ListBuffer.empty[String]
-    List(
-      (sparkConf.get(JARS_TO_DISTRIBUTE), LocalResourceType.FILE, true),
-      (sparkConf.get(FILES_TO_DISTRIBUTE), LocalResourceType.FILE, false),
-      (sparkConf.get(ARCHIVES_TO_DISTRIBUTE), LocalResourceType.ARCHIVE, false)
-    ).foreach { case (flist, resType, addToClasspath) =>
-      flist.foreach { file =>
-        val (_, localizedPath) = distribute(file, resType = resType)
-        // If addToClassPath, we ignore adding jar multiple times to distitrbuted cache.
-        if (addToClasspath) {
-          if (localizedPath != null) {
-            cachedSecondaryJarLinks += localizedPath
-          }
-        } else {
-          if (localizedPath == null) {
-            throw new IllegalArgumentException(s"Attempt to add ($file) multiple times" +
-              " to the distributed cache.")
-          }
-        }
-      }
-    }
-    if (cachedSecondaryJarLinks.nonEmpty) {
-      sparkConf.set(SECONDARY_JARS, cachedSecondaryJarLinks)
-    }
-
-    if (isClusterMode && args.primaryPyFile != null) {
-      distribute(args.primaryPyFile, appMasterOnly = true)
-    }
-
-    pySparkArchives.foreach { f => distribute(f) }
-
-    // The python files list needs to be treated especially. All files that are not an
-    // archive need to be placed in a subdirectory that will be added to PYTHONPATH.
-    sparkConf.get(PY_FILES).foreach { f =>
-      val targetDir = if (f.endsWith(".py")) Some(LOCALIZED_PYTHON_DIR) else None
-      distribute(f, targetDir = targetDir)
-    }
-
-    // Update the configuration with all the distributed files, minus the conf archive. The
-    // conf archive will be handled by the AM differently so that we avoid having to send
-    // this configuration by other means. See SPARK-14602 for one reason of why this is needed.
-    distCacheMgr.updateConfiguration(sparkConf)
-
-    // Upload the conf archive to HDFS manually, and record its location in the configuration.
-    // This will allow the AM to know where the conf archive is in HDFS, so that it can be
-    // distributed to the containers.
-    //
-    // This code forces the archive to be copied, so that unit tests pass (since in that case both
-    // file systems are the same and the archive wouldn't normally be copied). In most (all?)
-    // deployments, the archive would be copied anyway, since it's a temp file in the local file
-    // system.
-    val remoteConfArchivePath = new Path(destDir, LOCALIZED_CONF_ARCHIVE)
-    val remoteFs = FileSystem.get(remoteConfArchivePath.toUri(), hadoopConf)
-    sparkConf.set(CACHED_CONF_ARCHIVE, remoteConfArchivePath.toString())
-
-    val localConfArchive = new Path(createConfArchive().toURI())
-    copyFileToRemote(destDir, localConfArchive, replication, force = true,
-      destName = Some(LOCALIZED_CONF_ARCHIVE))
-
-    // Manually add the config archive to the cache manager so that the AM is launched with
-    // the proper files set up.
-    distCacheMgr.addResource(
-      remoteFs, hadoopConf, remoteConfArchivePath, localResources, LocalResourceType.ARCHIVE,
-      LOCALIZED_CONF_DIR, statCache, appMasterOnly = false)
-
-    // Clear the cache-related entries from the configuration to avoid them polluting the
-    // UI's environment page. This works for client mode; for cluster mode, this is handled
-    // by the AM.
-    CACHE_CONFIGS.foreach(sparkConf.remove)
-
-    localResources
-  }
-
-  /**
-   * Create an archive with the config files for distribution.
-   *
-   * These will be used by AM and executors. The files are zipped and added to the job as an
-   * archive, so that YARN will explode it when distributing to AM and executors. This directory
-   * is then added to the classpath of AM and executor process, just to make sure that everybody
-   * is using the same default config.
-   *
-   * This follows the order of precedence set by the startup scripts, in which HADOOP_CONF_DIR
-   * shows up in the classpath before YARN_CONF_DIR.
-   *
-   * Currently this makes a shallow copy of the conf directory. If there are cases where a
-   * Hadoop config directory contains subdirectories, this code will have to be fixed.
-   *
-   * The archive also contains some Spark configuration. Namely, it saves the contents of
-   * SparkConf in a file to be loaded by the AM process.
-   */
-  private def createConfArchive(): File = {
-    val hadoopConfFiles = new HashMap[String, File]()
-
-    // Uploading $SPARK_CONF_DIR/log4j.properties file to the distributed cache to make sure that
-    // the executors will use the latest configurations instead of the default values. This is
-    // required when user changes log4j.properties directly to set the log configurations. If
-    // configuration file is provided through --files then executors will be taking configurations
-    // from --files instead of $SPARK_CONF_DIR/log4j.properties.
-
-    // Also uploading metrics.properties to distributed cache if exists in classpath.
-    // If user specify this file using --files then executors will use the one
-    // from --files instead.
-    for { prop <- Seq("log4j.properties", "metrics.properties")
-          url <- Option(Utils.getContextOrSparkClassLoader.getResource(prop))
-          if url.getProtocol == "file" } {
-      hadoopConfFiles(prop) = new File(url.getPath)
-    }
-
-    Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey =>
-      sys.env.get(envKey).foreach { path =>
-        val dir = new File(path)
-        if (dir.isDirectory()) {
-          val files = dir.listFiles()
-          if (files == null) {
-            logWarning("Failed to list files under directory " + dir)
-          } else {
-            files.foreach { file =>
-              if (file.isFile && !hadoopConfFiles.contains(file.getName())) {
-                hadoopConfFiles(file.getName()) = file
-              }
-            }
-          }
-        }
-      }
-    }
-
-    val confArchive = File.createTempFile(LOCALIZED_CONF_DIR, ".zip",
-      new File(Utils.getLocalDir(sparkConf)))
-    val confStream = new ZipOutputStream(new FileOutputStream(confArchive))
-
-    try {
-      confStream.setLevel(0)
-      hadoopConfFiles.foreach { case (name, file) =>
-        if (file.canRead()) {
-          confStream.putNextEntry(new ZipEntry(name))
-          Files.copy(file, confStream)
-          confStream.closeEntry()
-        }
-      }
-
-      // Save Spark configuration to a file in the archive.
-      val props = new Properties()
-      sparkConf.getAll.foreach { case (k, v) => props.setProperty(k, v) }
-      confStream.putNextEntry(new ZipEntry(SPARK_CONF_FILE))
-      val writer = new OutputStreamWriter(confStream, StandardCharsets.UTF_8)
-      props.store(writer, "Spark configuration.")
-      writer.flush()
-      confStream.closeEntry()
-    } finally {
-      confStream.close()
-    }
-    confArchive
-  }
-
-  /**
-   * Set up the environment for launching our ApplicationMaster container.
-   */
-  private def setupLaunchEnv(
-      stagingDirPath: Path,
-      pySparkArchives: Seq[String]): HashMap[String, String] = {
-    logInfo("Setting up the launch environment for our AM container")
-    val env = new HashMap[String, String]()
-    populateClasspath(args, yarnConf, sparkConf, env, sparkConf.get(DRIVER_CLASS_PATH))
-    env("SPARK_YARN_MODE") = "true"
-    env("SPARK_YARN_STAGING_DIR") = stagingDirPath.toString
-    env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName()
-    if (loginFromKeytab) {
-      val credentialsFile = "credentials-" + UUID.randomUUID().toString
-      sparkConf.set(CREDENTIALS_FILE_PATH, new Path(stagingDirPath, credentialsFile).toString)
-      logInfo(s"Credentials file set to: $credentialsFile")
-    }
-
-    // Pick up any environment variables for the AM provided through spark.yarn.appMasterEnv.*
-    val amEnvPrefix = "spark.yarn.appMasterEnv."
-    sparkConf.getAll
-      .filter { case (k, v) => k.startsWith(amEnvPrefix) }
-      .map { case (k, v) => (k.substring(amEnvPrefix.length), v) }
-      .foreach { case (k, v) => YarnSparkHadoopUtil.addPathToEnvironment(env, k, v) }
-
-    // Keep this for backwards compatibility but users should move to the config
-    sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs =>
-    // Allow users to specify some environment variables.
-      YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs)
-      // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments.
-      env("SPARK_YARN_USER_ENV") = userEnvs
-    }
-
-    // If pyFiles contains any .py files, we need to add LOCALIZED_PYTHON_DIR to the PYTHONPATH
-    // of the container processes too. Add all non-.py files directly to PYTHONPATH.
-    //
-    // NOTE: the code currently does not handle .py files defined with a "local:" scheme.
-    val pythonPath = new ListBuffer[String]()
-    val (pyFiles, pyArchives) = sparkConf.get(PY_FILES).partition(_.endsWith(".py"))
-    if (pyFiles.nonEmpty) {
-      pythonPath += buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
-        LOCALIZED_PYTHON_DIR)
-    }
-    (pySparkArchives ++ pyArchives).foreach { path =>
-      val uri = Utils.resolveURI(path)
-      if (uri.getScheme != LOCAL_SCHEME) {
-        pythonPath += buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
-          new Path(uri).getName())
-      } else {
-        pythonPath += uri.getPath()
-      }
-    }
-
-    // Finally, update the Spark config to propagate PYTHONPATH to the AM and executors.
-    if (pythonPath.nonEmpty) {
-      val pythonPathStr = (sys.env.get("PYTHONPATH") ++ pythonPath)
-        .mkString(YarnSparkHadoopUtil.getClassPathSeparator)
-      env("PYTHONPATH") = pythonPathStr
-      sparkConf.setExecutorEnv("PYTHONPATH", pythonPathStr)
-    }
-
-    // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to
-    // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's
-    // SparkContext will not let that set spark* system properties, which is expected behavior for
-    // Yarn clients. So propagate it through the environment.
-    //
-    // Note that to warn the user about the deprecation in cluster mode, some code from
-    // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition
-    // described above).
-    if (isClusterMode) {
-      sys.env.get("SPARK_JAVA_OPTS").foreach { value =>
-        val warning =
-          s"""
-            |SPARK_JAVA_OPTS was detected (set to '$value').
-            |This is deprecated in Spark 1.0+.
-            |
-            |Please instead use:
-            | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application
-            | - ./spark-submit with --driver-java-options to set -X options for a driver
-            | - spark.executor.extraJavaOptions to set -X options for executors
-          """.stripMargin
-        logWarning(warning)
-        for (proc <- Seq("driver", "executor")) {
-          val key = s"spark.$proc.extraJavaOptions"
-          if (sparkConf.contains(key)) {
-            throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.")
-          }
-        }
-        env("SPARK_JAVA_OPTS") = value
-      }
-      // propagate PYSPARK_DRIVER_PYTHON and PYSPARK_PYTHON to driver in cluster mode
-      Seq("PYSPARK_DRIVER_PYTHON", "PYSPARK_PYTHON").foreach { envname =>
-        if (!env.contains(envname)) {
-          sys.env.get(envname).foreach(env(envname) = _)
-        }
-      }
-    }
-
-    sys.env.get(ENV_DIST_CLASSPATH).foreach { dcp =>
-      env(ENV_DIST_CLASSPATH) = dcp
-    }
-
-    env
-  }
-
-  /**
-   * Set up a ContainerLaunchContext to launch our ApplicationMaster container.
-   * This sets up the launch environment, java options, and the command for launching the AM.
-   */
-  private def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse)
-    : ContainerLaunchContext = {
-    logInfo("Setting up container launch context for our AM")
-    val appId = newAppResponse.getApplicationId
-    val appStagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId))
-    val pySparkArchives =
-      if (sparkConf.get(IS_PYTHON_APP)) {
-        findPySparkArchives()
-      } else {
-        Nil
-      }
-    val launchEnv = setupLaunchEnv(appStagingDirPath, pySparkArchives)
-    val localResources = prepareLocalResources(appStagingDirPath, pySparkArchives)
-
-    val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
-    amContainer.setLocalResources(localResources.asJava)
-    amContainer.setEnvironment(launchEnv.asJava)
-
-    val javaOpts = ListBuffer[String]()
-
-    // Set the environment variable through a command prefix
-    // to append to the existing value of the variable
-    var prefixEnv: Option[String] = None
-
-    // Add Xmx for AM memory
-    javaOpts += "-Xmx" + amMemory + "m"
-
-    val tmpDir = new Path(
-      YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
-      YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR
-    )
-    javaOpts += "-Djava.io.tmpdir=" + tmpDir
-
-    // TODO: Remove once cpuset version is pushed out.
-    // The context is, default gc for server class machines ends up using all cores to do gc -
-    // hence if there are multiple containers in same node, Spark GC affects all other containers'
-    // performance (which can be that of other Spark containers)
-    // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in
-    // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset
-    // of cores on a node.
-    val useConcurrentAndIncrementalGC = launchEnv.get("SPARK_USE_CONC_INCR_GC").exists(_.toBoolean)
-    if (useConcurrentAndIncrementalGC) {
-      // In our expts, using (default) throughput collector has severe perf ramifications in
-      // multi-tenant machines
-      javaOpts += "-XX:+UseConcMarkSweepGC"
-      javaOpts += "-XX:MaxTenuringThreshold=31"
-      javaOpts += "-XX:SurvivorRatio=8"
-      javaOpts += "-XX:+CMSIncrementalMode"
-      javaOpts += "-XX:+CMSIncrementalPacing"
-      javaOpts += "-XX:CMSIncrementalDutyCycleMin=0"
-      javaOpts += "-XX:CMSIncrementalDutyCycle=10"
-    }
-
-    // Include driver-specific java options if we are launching a driver
-    if (isClusterMode) {
-      val driverOpts = sparkConf.get(DRIVER_JAVA_OPTIONS).orElse(sys.env.get("SPARK_JAVA_OPTS"))
-      driverOpts.foreach { opts =>
-        javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
-      }
-      val libraryPaths = Seq(sparkConf.get(DRIVER_LIBRARY_PATH),
-        sys.props.get("spark.driver.libraryPath")).flatten
-      if (libraryPaths.nonEmpty) {
-        prefixEnv = Some(getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(libraryPaths)))
-      }
-      if (sparkConf.get(AM_JAVA_OPTIONS).isDefined) {
-        logWarning(s"${AM_JAVA_OPTIONS.key} will not take effect in cluster mode")
-      }
-    } else {
-      // Validate and include yarn am specific java options in yarn-client mode.
-      sparkConf.get(AM_JAVA_OPTIONS).foreach { opts =>
-        if (opts.contains("-Dspark")) {
-          val msg = s"${AM_JAVA_OPTIONS.key} is not allowed to set Spark options (was '$opts')."
-          throw new SparkException(msg)
-        }
-        if (opts.contains("-Xmx")) {
-          val msg = s"${AM_JAVA_OPTIONS.key} is not allowed to specify max heap memory settings " +
-            s"(was '$opts'). Use spark.yarn.am.memory instead."
-          throw new SparkException(msg)
-        }
-        javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
-      }
-      sparkConf.get(AM_LIBRARY_PATH).foreach { paths =>
-        prefixEnv = Some(getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(Seq(paths))))
-      }
-    }
-
-    // For log4j configuration to reference
-    javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR)
-    YarnCommandBuilderUtils.addPermGenSizeOpt(javaOpts)
-
-    val userClass =
-      if (isClusterMode) {
-        Seq("--class", YarnSparkHadoopUtil.escapeForShell(args.userClass))
-      } else {
-        Nil
-      }
-    val userJar =
-      if (args.userJar != null) {
-        Seq("--jar", args.userJar)
-      } else {
-        Nil
-      }
-    val primaryPyFile =
-      if (isClusterMode && args.primaryPyFile != null) {
-        Seq("--primary-py-file", new Path(args.primaryPyFile).getName())
-      } else {
-        Nil
-      }
-    val primaryRFile =
-      if (args.primaryRFile != null) {
-        Seq("--primary-r-file", args.primaryRFile)
-      } else {
-        Nil
-      }
-    val amClass =
-      if (isClusterMode) {
-        Utils.classForName("org.apache.spark.deploy.yarn.ApplicationMaster").getName
-      } else {
-        Utils.classForName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName
-      }
-    if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) {
-      args.userArgs = ArrayBuffer(args.primaryRFile) ++ args.userArgs
-    }
-    val userArgs = args.userArgs.flatMap { arg =>
-      Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg))
-    }
-    val amArgs =
-      Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ primaryRFile ++
-        userArgs ++ Seq(
-          "--properties-file", buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
-            LOCALIZED_CONF_DIR, SPARK_CONF_FILE))
-
-    // Command for the ApplicationMaster
-    val commands = prefixEnv ++ Seq(
-        YarnSparkHadoopUtil.expandEnvironment(Environment.JAVA_HOME) + "/bin/java", "-server"
-      ) ++
-      javaOpts ++ amArgs ++
-      Seq(
-        "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout",
-        "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
-
-    // TODO: it would be nicer to just make sure there are no null commands here
-    val printableCommands = commands.map(s => if (s == null) "null" else s).toList
-    amContainer.setCommands(printableCommands.asJava)
-
-    logDebug("===============================================================================")
-    logDebug("YARN AM launch context:")
-    logDebug(s"    user class: ${Option(args.userClass).getOrElse("N/A")}")
-    logDebug("    env:")
-    launchEnv.foreach { case (k, v) => logDebug(s"        $k -> $v") }
-    logDebug("    resources:")
-    localResources.foreach { case (k, v) => logDebug(s"        $k -> $v")}
-    logDebug("    command:")
-    logDebug(s"        ${printableCommands.mkString(" ")}")
-    logDebug("===============================================================================")
-
-    // send the acl settings into YARN to control who has access via YARN interfaces
-    val securityManager = new SecurityManager(sparkConf)
-    amContainer.setApplicationACLs(
-      YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager).asJava)
-    setupSecurityToken(amContainer)
-    amContainer
-  }
-
-  def setupCredentials(): Unit = {
-    loginFromKeytab = sparkConf.contains(PRINCIPAL.key)
-    if (loginFromKeytab) {
-      principal = sparkConf.get(PRINCIPAL).get
-      keytab = sparkConf.get(KEYTAB).orNull
-
-      require(keytab != null, "Keytab must be specified when principal is specified.")
-      logInfo("Attempting to login to the Kerberos" +
-        s" using principal: $principal and keytab: $keytab")
-      val f = new File(keytab)
-      // Generate a file name that can be used for the keytab file, that does not conflict
-      // with any user file.
-      val keytabFileName = f.getName + "-" + UUID.randomUUID().toString
-      sparkConf.set(KEYTAB.key, keytabFileName)
-      sparkConf.set(PRINCIPAL.key, principal)
-    }
-    // Defensive copy of the credentials
-    credentials = new Credentials(UserGroupInformation.getCurrentUser.getCredentials)
-  }
-
-  /**
-   * Report the state of an application until it has exited, either successfully or
-   * due to some failure, then return a pair of the yarn application state (FINISHED, FAILED,
-   * KILLED, or RUNNING) and the final application state (UNDEFINED, SUCCEEDED, FAILED,
-   * or KILLED).
-   *
-   * @param appId ID of the application to monitor.
-   * @param returnOnRunning Whether to also return the application state when it is RUNNING.
-   * @param logApplicationReport Whether to log details of the application report every iteration.
-   * @return A pair of the yarn application state and the final application state.
-   */
-  def monitorApplication(
-      appId: ApplicationId,
-      returnOnRunning: Boolean = false,
-      logApplicationReport: Boolean = true): (YarnApplicationState, FinalApplicationStatus) = {
-    val interval = sparkConf.get(REPORT_INTERVAL)
-    var lastState: YarnApplicationState = null
-    while (true) {
-      Thread.sleep(interval)
-      val report: ApplicationReport =
-        try {
-          getApplicationReport(appId)
-        } catch {
-          case e: ApplicationNotFoundException =>
-            logError(s"Application $appId not found.")
-            cleanupStagingDir(appId)
-            return (YarnApplicationState.KILLED, FinalApplicationStatus.KILLED)
-          case NonFatal(e) =>
-            logError(s"Failed to contact YARN for application $appId.", e)
-            // Don't necessarily clean up staging dir because status is unknown
-            return (YarnApplicationState.FAILED, FinalApplicationStatus.FAILED)
-        }
-      val state = report.getYarnApplicationState
-
-      if (logApplicationReport) {
-        logInfo(s"Application report for $appId (state: $state)")
-
-        // If DEBUG is enabled, log report details every iteration
-        // Otherwise, log them every time the application changes state
-        if (log.isDebugEnabled) {
-          logDebug(formatReportDetails(report))
-        } else if (lastState != state) {
-          logInfo(formatReportDetails(report))
-        }
-      }
-
-      if (lastState != state) {
-        state match {
-          case YarnApplicationState.RUNNING =>
-            reportLauncherState(SparkAppHandle.State.RUNNING)
-          case YarnApplicationState.FINISHED =>
-            report.getFinalApplicationStatus match {
-              case FinalApplicationStatus.FAILED =>
-                reportLauncherState(SparkAppHandle.State.FAILED)
-              case FinalApplicationStatus.KILLED =>
-                reportLauncherState(SparkAppHandle.State.KILLED)
-              case _ =>
-                reportLauncherState(SparkAppHandle.State.FINISHED)
-            }
-          case YarnApplicationState.FAILED =>
-            reportLauncherState(SparkAppHandle.State.FAILED)
-          case YarnApplicationState.KILLED =>
-            reportLauncherState(SparkAppHandle.State.KILLED)
-          case _ =>
-        }
-      }
-
-      if (state == YarnApplicationState.FINISHED ||
-        state == YarnApplicationState.FAILED ||
-        state == YarnApplicationState.KILLED) {
-        cleanupStagingDir(appId)
-        return (state, report.getFinalApplicationStatus)
-      }
-
-      if (returnOnRunning && state == YarnApplicationState.RUNNING) {
-        return (state, report.getFinalApplicationStatus)
-      }
-
-      lastState = state
-    }
-
-    // Never reached, but keeps compiler happy
-    throw new SparkException("While loop is depleted! This should never happen...")
-  }
-
-  private def formatReportDetails(report: ApplicationReport): String = {
-    val details = Seq[(String, String)](
-      ("client token", getClientToken(report)),
-      ("diagnostics", report.getDiagnostics),
-      ("ApplicationMaster host", report.getHost),
-      ("ApplicationMaster RPC port", report.getRpcPort.toString),
-      ("queue", report.getQueue),
-      ("start time", report.getStartTime.toString),
-      ("final status", report.getFinalApplicationStatus.toString),
-      ("tracking URL", report.getTrackingUrl),
-      ("user", report.getUser)
-    )
-
-    // Use more loggable format if value is null or empty
-    details.map { case (k, v) =>
-      val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A")
-      s"\n\t $k: $newValue"
-    }.mkString("")
-  }
-
-  /**
-   * Submit an application to the ResourceManager.
-   * If set spark.yarn.submit.waitAppCompletion to true, it will stay alive
-   * reporting the application's status until the application has exited for any reason.
-   * Otherwise, the client process will exit after submission.
-   * If the application finishes with a failed, killed, or undefined status,
-   * throw an appropriate SparkException.
-   */
-  def run(): Unit = {
-    this.appId = submitApplication()
-    if (!launcherBackend.isConnected() && fireAndForget) {
-      val report = getApplicationReport(appId)
-      val state = report.getYarnApplicationState
-      logInfo(s"Application report for $appId (state: $state)")
-      logInfo(formatReportDetails(report))
-      if (state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) {
-        throw new SparkException(s"Application $appId finished with status: $state")
-      }
-    } else {
-      val (yarnApplicationState, finalApplicationStatus) = monitorApplication(appId)
-      if (yarnApplicationState == YarnApplicationState.FAILED ||
-        finalApplicationStatus == FinalApplicationStatus.FAILED) {
-        throw new SparkException(s"Application $appId finished with failed status")
-      }
-      if (yarnApplicationState == YarnApplicationState.KILLED ||
-        finalApplicationStatus == FinalApplicationStatus.KILLED) {
-        throw new SparkException(s"Application $appId is killed")
-      }
-      if (finalApplicationStatus == FinalApplicationStatus.UNDEFINED) {
-        throw new SparkException(s"The final status of application $appId is undefined")
-      }
-    }
-  }
-
-  private def findPySparkArchives(): Seq[String] = {
-    sys.env.get("PYSPARK_ARCHIVES_PATH")
-      .map(_.split(",").toSeq)
-      .getOrElse {
-        val pyLibPath = Seq(sys.env("SPARK_HOME"), "python", "lib").mkString(File.separator)
-        val pyArchivesFile = new File(pyLibPath, "pyspark.zip")
-        require(pyArchivesFile.exists(),
-          s"$pyArchivesFile not found; cannot run pyspark application in YARN mode.")
-        val py4jFile = new File(pyLibPath, "py4j-0.10.4-src.zip")
-        require(py4jFile.exists(),
-          s"$py4jFile not found; cannot run pyspark application in YARN mode.")
-        Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath())
-      }
-  }
-
-}
-
-private object Client extends Logging {
-
-  def main(argStrings: Array[String]) {
-    if (!sys.props.contains("SPARK_SUBMIT")) {
-      logWarning("WARNING: This client is deprecated and will be removed in a " +
-        "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"")
-    }
-
-    // Set an env variable indicating we are running in YARN mode.
-    // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes
-    System.setProperty("SPARK_YARN_MODE", "true")
-    val sparkConf = new SparkConf
-    // SparkSubmit would use yarn cache to distribute files & jars in yarn mode,
-    // so remove them from sparkConf here for yarn mode.
-    sparkConf.remove("spark.jars")
-    sparkConf.remove("spark.files")
-    val args = new ClientArguments(argStrings)
-    new Client(args, sparkConf).run()
-  }
-
-  // Alias for the user jar
-  val APP_JAR_NAME: String = "__app__.jar"
-
-  // URI scheme that identifies local resources
-  val LOCAL_SCHEME = "local"
-
-  // Staging directory for any temporary jars or files
-  val SPARK_STAGING: String = ".sparkStaging"
-
-
-  // Staging directory is private! -> rwx--------
-  val STAGING_DIR_PERMISSION: FsPermission =
-    FsPermission.createImmutable(Integer.parseInt("700", 8).toShort)
-
-  // App files are world-wide readable and owner writable -> rw-r--r--
-  val APP_FILE_PERMISSION: FsPermission =
-    FsPermission.createImmutable(Integer.parseInt("644", 8).toShort)
-
-  // Distribution-defined classpath to add to processes
-  val ENV_DIST_CLASSPATH = "SPARK_DIST_CLASSPATH"
-
-  // Subdirectory where the user's Spark and Hadoop config files will be placed.
-  val LOCALIZED_CONF_DIR = "__spark_conf__"
-
-  // File containing the conf archive in the AM. See prepareLocalResources().
-  val LOCALIZED_CONF_ARCHIVE = LOCALIZED_CONF_DIR + ".zip"
-
-  // Name of the file in the conf archive containing Spark configuration.
-  val SPARK_CONF_FILE = "__spark_conf__.properties"
-
-  // Subdirectory where the user's python files (not archives) will be placed.
-  val LOCALIZED_PYTHON_DIR = "__pyfiles__"
-
-  // Subdirectory where Spark libraries will be placed.
-  val LOCALIZED_LIB_DIR = "__spark_libs__"
-
-  /**
-   * Return the path to the given application's staging directory.
-   */
-  private def getAppStagingDir(appId: ApplicationId): String = {
-    buildPath(SPARK_STAGING, appId.toString())
-  }
-
-  /**
-   * Populate the classpath entry in the given environment map with any application
-   * classpath specified through the Hadoop and Yarn configurations.
-   */
-  private[yarn] def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String])
-    : Unit = {
-    val classPathElementsToAdd = getYarnAppClasspath(conf) ++ getMRAppClasspath(conf)
-    for (c <- classPathElementsToAdd.flatten) {
-      YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, c.trim)
-    }
-  }
-
-  private def getYarnAppClasspath(conf: Configuration): Option[Seq[String]] =
-    Option(conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) match {
-      case Some(s) => Some(s.toSeq)
-      case None => getDefaultYarnApplicationClasspath
-    }
-
-  private def getMRAppClasspath(conf: Configuration): Option[Seq[String]] =
-    Option(conf.getStrings("mapreduce.application.classpath")) match {
-      case Some(s) => Some(s.toSeq)
-      case None => getDefaultMRApplicationClasspath
-    }
-
-  private[yarn] def getDefaultYarnApplicationClasspath: Option[Seq[String]] = {
-    val triedDefault = Try[Seq[String]] {
-      val field = classOf[YarnConfiguration].getField("DEFAULT_YARN_APPLICATION_CLASSPATH")
-      val value = field.get(null).asInstanceOf[Array[String]]
-      value.toSeq
-    } recoverWith {
-      case e: NoSuchFieldException => Success(Seq.empty[String])
-    }
-
-    triedDefault match {
-      case f: Failure[_] =>
-        logError("Unable to obtain the default YARN Application classpath.", f.exception)
-      case s: Success[Seq[String]] =>
-        logDebug(s"Using the default YARN application classpath: ${s.get.mkString(",")}")
-    }
-
-    triedDefault.toOption
-  }
-
-  private[yarn] def getDefaultMRApplicationClasspath: Option[Seq[String]] = {
-    val triedDefault = Try[Seq[String]] {
-      val field = classOf[MRJobConfig].getField("DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH")
-      StringUtils.getStrings(field.get(null).asInstanceOf[String]).toSeq
-    } recoverWith {
-      case e: NoSuchFieldException => Success(Seq.empty[String])
-    }
-
-    triedDefault match {
-      case f: Failure[_] =>
-        logError("Unable to obtain the default MR Application classpath.", f.exception)
-      case s: Success[Seq[String]] =>
-        logDebug(s"Using the default MR application classpath: ${s.get.mkString(",")}")
-    }
-
-    triedDefault.toOption
-  }
-
-  /**
-   * Populate the classpath entry in the given environment map.
-   *
-   * User jars are generally not added to the JVM's system classpath; those are handled by the AM
-   * and executor backend. When the deprecated `spark.yarn.user.classpath.first` is used, user jars
-   * are included in the system classpath, though. The extra class path and other uploaded files are
-   * always made available through the system class path.
-   *
-   * @param args Client arguments (when starting the AM) or null (when starting executors).
-   */
-  private[yarn] def populateClasspath(
-      args: ClientArguments,
-      conf: Configuration,
-      sparkConf: SparkConf,
-      env: HashMap[String, String],
-      extraClassPath: Option[String] = None): Unit = {
-    extraClassPath.foreach { cp =>
-      addClasspathEntry(getClusterPath(sparkConf, cp), env)
-    }
-
-    addClasspathEntry(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), env)
-
-    addClasspathEntry(
-      YarnSparkHadoopUtil.expandEnvironment(Environment.PWD) + Path.SEPARATOR +
-        LOCALIZED_CONF_DIR, env)
-
-    if (sparkConf.get(USER_CLASS_PATH_FIRST)) {
-      // in order to properly add the app jar when user classpath is first
-      // we have to do the mainJar separate in order to send the right thing
-      // into addFileToClasspath
-      val mainJar =
-        if (args != null) {
-          getMainJarUri(Option(args.userJar))
-        } else {
-          getMainJarUri(sparkConf.get(APP_JAR))
-        }
-      mainJar.foreach(addFileToClasspath(sparkConf, conf, _, APP_JAR_NAME, env))
-
-      val secondaryJars =
-        if (args != null) {
-          getSecondaryJarUris(Option(sparkConf.get(JARS_TO_DISTRIBUTE)))
-        } else {
-          getSecondaryJarUris(sparkConf.get(SECONDARY_JARS))
-        }
-      secondaryJars.foreach { x =>
-        addFileToClasspath(sparkConf, conf, x, null, env)
-      }
-    }
-
-    // Add the Spark jars to the classpath, depending on how they were distributed.
-    addClasspathEntry(buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
-      LOCALIZED_LIB_DIR, "*"), env)
-    if (!sparkConf.get(SPARK_ARCHIVE).isDefined) {
-      sparkConf.get(SPARK_JARS).foreach { jars =>
-        jars.filter(isLocalUri).foreach { jar =>
-          addClasspathEntry(getClusterPath(sparkConf, jar), env)
-        }
-      }
-    }
-
-    populateHadoopClasspath(conf, env)
-    sys.env.get(ENV_DIST_CLASSPATH).foreach { cp =>
-      addClasspathEntry(getClusterPath(sparkConf, cp), env)
-    }
-  }
-
-  /**
-   * Returns a list of URIs representing the user classpath.
-   *
-   * @param conf Spark configuration.
-   */
-  def getUserClasspath(conf: SparkConf): Array[URI] = {
-    val mainUri = getMainJarUri(conf.get(APP_JAR))
-    val secondaryUris = getSecondaryJarUris(conf.get(SECONDARY_JARS))
-    (mainUri ++ secondaryUris).toArray
-  }
-
-  private def getMainJarUri(mainJar: Option[String]): Option[URI] = {
-    mainJar.flatMap { path =>
-      val uri = Utils.resolveURI(path)
-      if (uri.getScheme == LOCAL_SCHEME) Some(uri) else None
-    }.orElse(Some(new URI(APP_JAR_NAME)))
-  }
-
-  private def getSecondaryJarUris(secondaryJars: Option[Seq[String]]): Seq[URI] = {
-    secondaryJars.getOrElse(Nil).map(new URI(_))
-  }
-
-  /**
-   * Adds the given path to the classpath, handling "local:" URIs correctly.
-   *
-   * If an alternate name for the file is given, and it's not a "local:" file, the alternate
-   * name will be added to the classpath (relative to the job's work directory).
-   *
-   * If not a "local:" file and no alternate name, the linkName will be added to the classpath.
-   *
-   * @param conf        Spark configuration.
-   * @param hadoopConf  Hadoop configuration.
-   * @param uri         URI to add to classpath (optional).
-   * @param fileName    Alternate name for the file (optional).
-   * @param env         Map holding the environment variables.
-   */
-  private def addFileToClasspath(
-      conf: SparkConf,
-      hadoopConf: Configuration,
-      uri: URI,
-      fileName: String,
-      env: HashMap[String, String]): Unit = {
-    if (uri != null && uri.getScheme == LOCAL_SCHEME) {
-      addClasspathEntry(getClusterPath(conf, uri.getPath), env)
-    } else if (fileName != null) {
-      addClasspathEntry(buildPath(
-        YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), fileName), env)
-    } else if (uri != null) {
-      val localPath = getQualifiedLocalPath(uri, hadoopConf)
-      val linkName = Option(uri.getFragment()).getOrElse(localPath.getName())
-      addClasspathEntry(buildPath(
-        YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), linkName), env)
-    }
-  }
-
-  /**
-   * Add the given path to the classpath entry of the given environment map.
-   * If the classpath is already set, this appends the new path to the existing classpath.
-   */
-  private def addClasspathEntry(path: String, env: HashMap[String, String]): Unit =
-    YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, path)
-
-  /**
-   * Returns the path to be sent to the NM for a path that is valid on the gateway.
-   *
-   * This method uses two configuration values:
-   *
-   *  - spark.yarn.config.gatewayPath: a string that identifies a portion of the input path that may
-   *    only be valid in the gateway node.
-   *  - spark.yarn.config.replacementPath: a string with which to replace the gateway path. This may
-   *    contain, for example, env variable references, which will be expanded by the NMs when
-   *    starting containers.
-   *
-   * If either config is not available, the input path is returned.
-   */
-  def getClusterPath(conf: SparkConf, path: String): String = {
-    val localPath = conf.get(GATEWAY_ROOT_PATH)
-    val clusterPath = conf.get(REPLACEMENT_ROOT_PATH)
-    if (localPath != null && clusterPath != null) {
-      path.replace(localPath, clusterPath)
-    } else {
-      path
-    }
-  }
-
-  /**
-   * Return whether the two file systems are the same.
-   */
-  private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = {
-    val srcUri = srcFs.getUri()
-    val dstUri = destFs.getUri()
-    if (srcUri.getScheme() == null || srcUri.getScheme() != dstUri.getScheme()) {
-      return false
-    }
-
-    var srcHost = srcUri.getHost()
-    var dstHost = dstUri.getHost()
-
-    // In HA or when using viewfs, the host part of the URI may not actually be a host, but the
-    // name of the HDFS namespace. Those names won't resolve, so avoid even trying if they
-    // match.
-    if (srcHost != null && dstHost != null && srcHost != dstHost) {
-      try {
-        srcHost = InetAddress.getByName(srcHost).getCanonicalHostName()
-        dstHost = InetAddress.getByName(dstHost).getCanonicalHostName()
-      } catch {
-        case e: UnknownHostException =>
-          return false
-      }
-    }
-
-    Objects.equal(srcHost, dstHost) && srcUri.getPort() == dstUri.getPort()
-  }
-
-  /**
-   * Given a local URI, resolve it and return a qualified local path that corresponds to the URI.
-   * This is used for preparing local resources to be included in the container launch context.
-   */
-  private def getQualifiedLocalPath(localURI: URI, hadoopConf: Configuration): Path = {
-    val qualifiedURI =
-      if (localURI.getScheme == null) {
-        // If not specified, assume this is in the local filesystem to keep the behavior
-        // consistent with that of Hadoop
-        new URI(FileSystem.getLocal(hadoopConf).makeQualified(new Path(localURI)).toString)
-      } else {
-        localURI
-      }
-    new Path(qualifiedURI)
-  }
-
-  /**
-   * Whether to consider jars provided by the user to have precedence over the Spark jars when
-   * loading user classes.
-   */
-  def isUserClassPathFirst(conf: SparkConf, isDriver: Boolean): Boolean = {
-    if (isDriver) {
-      conf.get(DRIVER_USER_CLASS_PATH_FIRST)
-    } else {
-      conf.get(EXECUTOR_USER_CLASS_PATH_FIRST)
-    }
-  }
-
-  /**
-   * Joins all the path components using Path.SEPARATOR.
-   */
-  def buildPath(components: String*): String = {
-    components.mkString(Path.SEPARATOR)
-  }
-
-  /** Returns whether the URI is a "local:" URI. */
-  def isLocalUri(uri: String): Boolean = {
-    uri.startsWith(s"$LOCAL_SCHEME:")
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
deleted file mode 100644
index 61c027e..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ /dev/null
@@ -1,86 +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.deploy.yarn
-
-import scala.collection.mutable.ArrayBuffer
-
-// TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware !
-private[spark] class ClientArguments(args: Array[String]) {
-
-  var userJar: String = null
-  var userClass: String = null
-  var primaryPyFile: String = null
-  var primaryRFile: String = null
-  var userArgs: ArrayBuffer[String] = new ArrayBuffer[String]()
-
-  parseArgs(args.toList)
-
-  private def parseArgs(inputArgs: List[String]): Unit = {
-    var args = inputArgs
-
-    while (!args.isEmpty) {
-      args match {
-        case ("--jar") :: value :: tail =>
-          userJar = value
-          args = tail
-
-        case ("--class") :: value :: tail =>
-          userClass = value
-          args = tail
-
-        case ("--primary-py-file") :: value :: tail =>
-          primaryPyFile = value
-          args = tail
-
-        case ("--primary-r-file") :: value :: tail =>
-          primaryRFile = value
-          args = tail
-
-        case ("--arg") :: value :: tail =>
-          userArgs += value
-          args = tail
-
-        case Nil =>
-
-        case _ =>
-          throw new IllegalArgumentException(getUsageMessage(args))
-      }
-    }
-
-    if (primaryPyFile != null && primaryRFile != null) {
-      throw new IllegalArgumentException("Cannot have primary-py-file and primary-r-file" +
-        " at the same time")
-    }
-  }
-
-  private def getUsageMessage(unknownParam: List[String] = null): String = {
-    val message = if (unknownParam != null) s"Unknown/unsupported param $unknownParam\n" else ""
-    message +
-      s"""
-      |Usage: org.apache.spark.deploy.yarn.Client [options]
-      |Options:
-      |  --jar JAR_PATH           Path to your application's JAR file (required in yarn-cluster
-      |                           mode)
-      |  --class CLASS_NAME       Name of your application's main class (required)
-      |  --primary-py-file        A main Python file
-      |  --primary-r-file         A main R file
-      |  --arg ARG                Argument to be passed to your application's main class.
-      |                           Multiple invocations are possible, each will be passed in order.
-      """.stripMargin
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
deleted file mode 100644
index dcc2288..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
+++ /dev/null
@@ -1,186 +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.deploy.yarn
-
-import java.net.URI
-
-import scala.collection.mutable.{HashMap, ListBuffer, Map}
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
-import org.apache.hadoop.fs.permission.FsAction
-import org.apache.hadoop.yarn.api.records._
-import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
-
-import org.apache.spark.SparkConf
-import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.internal.Logging
-
-private case class CacheEntry(
-  uri: URI,
-  size: Long,
-  modTime: Long,
-  visibility: LocalResourceVisibility,
-  resType: LocalResourceType)
-
-/** Client side methods to setup the Hadoop distributed cache */
-private[spark] class ClientDistributedCacheManager() extends Logging {
-
-  private val distCacheEntries = new ListBuffer[CacheEntry]()
-
-  /**
-   * Add a resource to the list of distributed cache resources. This list can
-   * be sent to the ApplicationMaster and possibly the executors so that it can
-   * be downloaded into the Hadoop distributed cache for use by this application.
-   * Adds the LocalResource to the localResources HashMap passed in and saves
-   * the stats of the resources to they can be sent to the executors and verified.
-   *
-   * @param fs FileSystem
-   * @param conf Configuration
-   * @param destPath path to the resource
-   * @param localResources localResource hashMap to insert the resource into
-   * @param resourceType LocalResourceType
-   * @param link link presented in the distributed cache to the destination
-   * @param statCache cache to store the file/directory stats
-   * @param appMasterOnly Whether to only add the resource to the app master
-   */
-  def addResource(
-      fs: FileSystem,
-      conf: Configuration,
-      destPath: Path,
-      localResources: HashMap[String, LocalResource],
-      resourceType: LocalResourceType,
-      link: String,
-      statCache: Map[URI, FileStatus],
-      appMasterOnly: Boolean = false): Unit = {
-    val destStatus = fs.getFileStatus(destPath)
-    val amJarRsrc = Records.newRecord(classOf[LocalResource])
-    amJarRsrc.setType(resourceType)
-    val visibility = getVisibility(conf, destPath.toUri(), statCache)
-    amJarRsrc.setVisibility(visibility)
-    amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(destPath))
-    amJarRsrc.setTimestamp(destStatus.getModificationTime())
-    amJarRsrc.setSize(destStatus.getLen())
-    require(link != null && link.nonEmpty, "You must specify a valid link name.")
-    localResources(link) = amJarRsrc
-
-    if (!appMasterOnly) {
-      val uri = destPath.toUri()
-      val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link)
-      distCacheEntries += CacheEntry(pathURI, destStatus.getLen(), destStatus.getModificationTime(),
-        visibility, resourceType)
-    }
-  }
-
-  /**
-   * Writes down information about cached files needed in executors to the given configuration.
-   */
-  def updateConfiguration(conf: SparkConf): Unit = {
-    conf.set(CACHED_FILES, distCacheEntries.map(_.uri.toString))
-    conf.set(CACHED_FILES_SIZES, distCacheEntries.map(_.size))
-    conf.set(CACHED_FILES_TIMESTAMPS, distCacheEntries.map(_.modTime))
-    conf.set(CACHED_FILES_VISIBILITIES, distCacheEntries.map(_.visibility.name()))
-    conf.set(CACHED_FILES_TYPES, distCacheEntries.map(_.resType.name()))
-  }
-
-  /**
-   * Returns the local resource visibility depending on the cache file permissions
-   * @return LocalResourceVisibility
-   */
-  private[yarn] def getVisibility(
-      conf: Configuration,
-      uri: URI,
-      statCache: Map[URI, FileStatus]): LocalResourceVisibility = {
-    if (isPublic(conf, uri, statCache)) {
-      LocalResourceVisibility.PUBLIC
-    } else {
-      LocalResourceVisibility.PRIVATE
-    }
-  }
-
-  /**
-   * Returns a boolean to denote whether a cache file is visible to all (public)
-   * @return true if the path in the uri is visible to all, false otherwise
-   */
-  private def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = {
-    val fs = FileSystem.get(uri, conf)
-    val current = new Path(uri.getPath())
-    // the leaf level file should be readable by others
-    if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) {
-      return false
-    }
-    ancestorsHaveExecutePermissions(fs, current.getParent(), statCache)
-  }
-
-  /**
-   * Returns true if all ancestors of the specified path have the 'execute'
-   * permission set for all users (i.e. that other users can traverse
-   * the directory hierarchy to the given path)
-   * @return true if all ancestors have the 'execute' permission set for all users
-   */
-  private def ancestorsHaveExecutePermissions(
-      fs: FileSystem,
-      path: Path,
-      statCache: Map[URI, FileStatus]): Boolean = {
-    var current = path
-    while (current != null) {
-      // the subdirs in the path should have execute permissions for others
-      if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE, statCache)) {
-        return false
-      }
-      current = current.getParent()
-    }
-    true
-  }
-
-  /**
-   * Checks for a given path whether the Other permissions on it
-   * imply the permission in the passed FsAction
-   * @return true if the path in the uri is visible to all, false otherwise
-   */
-  private def checkPermissionOfOther(
-      fs: FileSystem,
-      path: Path,
-      action: FsAction,
-      statCache: Map[URI, FileStatus]): Boolean = {
-    val status = getFileStatus(fs, path.toUri(), statCache)
-    val perms = status.getPermission()
-    val otherAction = perms.getOtherAction()
-    otherAction.implies(action)
-  }
-
-  /**
-   * Checks to see if the given uri exists in the cache, if it does it
-   * returns the existing FileStatus, otherwise it stats the uri, stores
-   * it in the cache, and returns the FileStatus.
-   * @return FileStatus
-   */
-  private[yarn] def getFileStatus(
-      fs: FileSystem,
-      uri: URI,
-      statCache: Map[URI, FileStatus]): FileStatus = {
-    val stat = statCache.get(uri) match {
-      case Some(existstat) => existstat
-      case None =>
-        val newStat = fs.getFileStatus(new Path(uri))
-        statCache.put(uri, newStat)
-        newStat
-    }
-    stat
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
deleted file mode 100644
index 868c2ed..0000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
+++ /dev/null
@@ -1,266 +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.deploy.yarn
-
-import java.io.File
-import java.nio.ByteBuffer
-import java.util.Collections
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.{HashMap, ListBuffer}
-
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.io.DataOutputBuffer
-import org.apache.hadoop.security.UserGroupInformation
-import org.apache.hadoop.yarn.api._
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-import org.apache.hadoop.yarn.api.records._
-import org.apache.hadoop.yarn.client.api.NMClient
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.yarn.ipc.YarnRPC
-import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
-
-import org.apache.spark.{SecurityManager, SparkConf, SparkException}
-import org.apache.spark.internal.Logging
-import org.apache.spark.internal.config._
-import org.apache.spark.launcher.YarnCommandBuilderUtils
-import org.apache.spark.network.util.JavaUtils
-import org.apache.spark.util.Utils
-
-private[yarn] class ExecutorRunnable(
-    container: Option[Container],
-    conf: YarnConfiguration,
-    sparkConf: SparkConf,
-    masterAddress: String,
-    executorId: String,
-    hostname: String,
-    executorMemory: Int,
-    executorCores: Int,
-    appId: String,
-    securityMgr: SecurityManager,
-    localResources: Map[String, LocalResource]) extends Logging {
-
-  var rpc: YarnRPC = YarnRPC.create(conf)
-  var nmClient: NMClient = _
-
-  def run(): Unit = {
-    logDebug("Starting Executor Container")
-    nmClient = NMClient.createNMClient()
-    nmClient.init(conf)
-    nmClient.start()
-    startContainer()
-  }
-
-  def launchContextDebugInfo(): String = {
-    val commands = prepareCommand()
-    val env = prepareEnvironment()
-
-    s"""
-    |===============================================================================
-    |YARN executor launch context:
-    |  env:
-    |${Utils.redact(sparkConf, env.toSeq).map { case (k, v) => s"    $k -> $v\n" }.mkString}
-    |  command:
-    |    ${commands.mkString(" \\ \n      ")}
-    |
-    |  resources:
-    |${localResources.map { case (k, v) => s"    $k -> $v\n" }.mkString}
-    |===============================================================================""".stripMargin
-  }
-
-  def startContainer(): java.util.Map[String, ByteBuffer] = {
-    val ctx = Records.newRecord(classOf[ContainerLaunchContext])
-      .asInstanceOf[ContainerLaunchContext]
-    val env = prepareEnvironment().asJava
-
-    ctx.setLocalResources(localResources.asJava)
-    ctx.setEnvironment(env)
-
-    val credentials = UserGroupInformation.getCurrentUser().getCredentials()
-    val dob = new DataOutputBuffer()
-    credentials.writeTokenStorageToStream(dob)
-    ctx.setTokens(ByteBuffer.wrap(dob.getData()))
-
-    val commands = prepareCommand()
-
-    ctx.setCommands(commands.asJava)
-    ctx.setApplicationACLs(
-      YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr).asJava)
-
-    // If external shuffle service is enabled, register with the Yarn shuffle service already
-    // started on the NodeManager and, if authentication is enabled, provide it with our secret
-    // key for fetching shuffle files later
-    if (sparkConf.get(SHUFFLE_SERVICE_ENABLED)) {
-      val secretString = securityMgr.getSecretKey()
-      val secretBytes =
-        if (secretString != null) {
-          // This conversion must match how the YarnShuffleService decodes our secret
-          JavaUtils.stringToBytes(secretString)
-        } else {
-          // Authentication is not enabled, so just provide dummy metadata
-          ByteBuffer.allocate(0)
-        }
-      ctx.setServiceData(Collections.singletonMap("spark_shuffle", secretBytes))
-    }
-
-    // Send the start request to the ContainerManager
-    try {
-      nmClient.startContainer(container.get, ctx)
-    } catch {
-      case ex: Exception =>
-        throw new SparkException(s"Exception while starting container ${container.get.getId}" +
-          s" on host $hostname", ex)
-    }
-  }
-
-  private def prepareCommand(): List[String] = {
-    // Extra options for the JVM
-    val javaOpts = ListBuffer[String]()
-
-    // Set the environment variable through a command prefix
-    // to append to the existing value of the variable
-    var prefixEnv: Option[String] = None
-
-    // Set the JVM memory
-    val executorMemoryString = executorMemory + "m"
-    javaOpts += "-Xmx" + executorMemoryString
-
-    // Set extra Java options for the executor, if defined
-    sparkConf.get(EXECUTOR_JAVA_OPTIONS).foreach { opts =>
-      javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
-    }
-    sys.env.get("SPARK_JAVA_OPTS").foreach { opts =>
-      javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
-    }
-    sparkConf.get(EXECUTOR_LIBRARY_PATH).foreach { p =>
-      prefixEnv = Some(Client.getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(Seq(p))))
-    }
-
-    javaOpts += "-Djava.io.tmpdir=" +
-      new Path(
-        YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
-        YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR
-      )
-
-    // Certain configs need to be passed here because they are needed before the Executor
-    // registers with the Scheduler and transfers the spark configs. Since the Executor backend
-    // uses RPC to connect to the scheduler, the RPC settings are needed as well as the
-    // authentication settings.
-    sparkConf.getAll
-      .filter { case (k, v) => SparkConf.isExecutorStartupConf(k) }
-      .foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") }
-
-    // Commenting it out for now - so that people can refer to the properties if required. Remove
-    // it once cpuset version is pushed out.
-    // The context is, default gc for server class machines end up using all cores to do gc - hence
-    // if there are multiple containers in same node, spark gc effects all other containers
-    // performance (which can also be other spark containers)
-    // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
-    // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
-    // of cores on a node.
-    /*
-        else {
-          // If no java_opts specified, default to using -XX:+CMSIncrementalMode
-          // It might be possible that other modes/config is being done in
-          // spark.executor.extraJavaOptions, so we don't want to mess with it.
-          // In our expts, using (default) throughput collector has severe perf ramifications in
-          // multi-tenant machines
-          // The options are based on
-          // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use
-          // %20the%20Concurrent%20Low%20Pause%20Collector|outline
-          javaOpts += "-XX:+UseConcMarkSweepGC"
-          javaOpts += "-XX:+CMSIncrementalMode"
-          javaOpts += "-XX:+CMSIncrementalPacing"
-          javaOpts += "-XX:CMSIncrementalDutyCycleMin=0"
-          javaOpts += "-XX:CMSIncrementalDutyCycle=10"
-        }
-    */
-
-    // For log4j configuration to reference
-    javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR)
-    YarnCommandBuilderUtils.addPermGenSizeOpt(javaOpts)
-
-    val userClassPath = Client.getUserClasspath(sparkConf).flatMap { uri =>
-      val absPath =
-        if (new File(uri.getPath()).isAbsolute()) {
-          Client.getClusterPath(sparkConf, uri.getPath())
-        } else {
-          Client.buildPath(Environment.PWD.$(), uri.getPath())
-        }
-      Seq("--user-class-path", "file:" + absPath)
-    }.toSeq
-
-    YarnSparkHadoopUtil.addOutOfMemoryErrorArgument(javaOpts)
-    val commands = prefixEnv ++ Seq(
-      YarnSparkHadoopUtil.expandEnvironment(Environment.JAVA_HOME) + "/bin/java",
-      "-server") ++
-      javaOpts ++
-      Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend",
-        "--driver-url", masterAddress,
-        "--executor-id", executorId,
-        "--hostname", hostname,
-        "--cores", executorCores.toString,
-        "--app-id", appId) ++
-      userClassPath ++
-      Seq(
-        s"1>${ApplicationConstants.LOG_DIR_EXPANSION_VAR}/stdout",
-        s"2>${ApplicationConstants.LOG_DIR_EXPANSION_VAR}/stderr")
-
-    // TODO: it would be nicer to just make sure there are no null commands here
-    commands.map(s => if (s == null) "null" else s).toList
-  }
-
-  private def prepareEnvironment(): HashMap[String, String] = {
-    val env = new HashMap[String, String]()
-    Client.populateClasspath(null, conf, sparkConf, env, sparkConf.get(EXECUTOR_CLASS_PATH))
-
-    sparkConf.getExecutorEnv.foreach { case (key, value) =>
-      // This assumes each executor environment variable set here is a path
-      // This is kept for backward compatibility and consistency with hadoop
-      YarnSparkHadoopUtil.addPathToEnvironment(env, key, value)
-    }
-
-    // Keep this for backwards compatibility but users should move to the config
-    sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs =>
-      YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs)
-    }
-
-    // lookup appropriate http scheme for container log urls
-    val yarnHttpPolicy = conf.get(
-      YarnConfiguration.YARN_HTTP_POLICY_KEY,
-      YarnConfiguration.YARN_HTTP_POLICY_DEFAULT
-    )
-    val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://"
-
-    // Add log urls
-    container.foreach { c =>
-      sys.env.get("SPARK_USER").foreach { user =>
-        val containerId = ConverterUtils.toString(c.getId)
-        val address = c.getNodeHttpAddress
-        val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user"
-
-        env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=-4096"
-        env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=-4096"
-      }
-    }
-
-    System.getenv().asScala.filterKeys(_.startsWith("SPARK"))
-      .foreach { case (k, v) => env(k) = v }
-    env
-  }
-}


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


[16/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
deleted file mode 100644
index 1d742fe..0000000
--- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
+++ /dev/null
@@ -1,524 +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.mesos
-
-import java.util.{List => JList}
-import java.util.concurrent.CountDownLatch
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-import scala.util.control.NonFatal
-
-import com.google.common.base.Splitter
-import org.apache.mesos.{MesosSchedulerDriver, Protos, Scheduler, SchedulerDriver}
-import org.apache.mesos.Protos.{TaskState => MesosTaskState, _}
-import org.apache.mesos.Protos.FrameworkInfo.Capability
-import org.apache.mesos.protobuf.{ByteString, GeneratedMessage}
-
-import org.apache.spark.{SparkConf, SparkContext, SparkException}
-import org.apache.spark.TaskState
-import org.apache.spark.internal.Logging
-import org.apache.spark.internal.config._
-import org.apache.spark.util.Utils
-
-
-
-/**
- * Shared trait for implementing a Mesos Scheduler. This holds common state and helper
- * methods and Mesos scheduler will use.
- */
-trait MesosSchedulerUtils extends Logging {
-  // Lock used to wait for scheduler to be registered
-  private final val registerLatch = new CountDownLatch(1)
-
-  // Driver for talking to Mesos
-  protected var mesosDriver: SchedulerDriver = null
-
-  /**
-   * Creates a new MesosSchedulerDriver that communicates to the Mesos master.
-   *
-   * @param masterUrl The url to connect to Mesos master
-   * @param scheduler the scheduler class to receive scheduler callbacks
-   * @param sparkUser User to impersonate with when running tasks
-   * @param appName The framework name to display on the Mesos UI
-   * @param conf Spark configuration
-   * @param webuiUrl The WebUI url to link from Mesos UI
-   * @param checkpoint Option to checkpoint tasks for failover
-   * @param failoverTimeout Duration Mesos master expect scheduler to reconnect on disconnect
-   * @param frameworkId The id of the new framework
-   */
-  protected def createSchedulerDriver(
-      masterUrl: String,
-      scheduler: Scheduler,
-      sparkUser: String,
-      appName: String,
-      conf: SparkConf,
-      webuiUrl: Option[String] = None,
-      checkpoint: Option[Boolean] = None,
-      failoverTimeout: Option[Double] = None,
-      frameworkId: Option[String] = None): SchedulerDriver = {
-    val fwInfoBuilder = FrameworkInfo.newBuilder().setUser(sparkUser).setName(appName)
-    val credBuilder = Credential.newBuilder()
-    webuiUrl.foreach { url => fwInfoBuilder.setWebuiUrl(url) }
-    checkpoint.foreach { checkpoint => fwInfoBuilder.setCheckpoint(checkpoint) }
-    failoverTimeout.foreach { timeout => fwInfoBuilder.setFailoverTimeout(timeout) }
-    frameworkId.foreach { id =>
-      fwInfoBuilder.setId(FrameworkID.newBuilder().setValue(id).build())
-    }
-    fwInfoBuilder.setHostname(Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(
-      conf.get(DRIVER_HOST_ADDRESS)))
-    conf.getOption("spark.mesos.principal").foreach { principal =>
-      fwInfoBuilder.setPrincipal(principal)
-      credBuilder.setPrincipal(principal)
-    }
-    conf.getOption("spark.mesos.secret").foreach { secret =>
-      credBuilder.setSecret(secret)
-    }
-    if (credBuilder.hasSecret && !fwInfoBuilder.hasPrincipal) {
-      throw new SparkException(
-        "spark.mesos.principal must be configured when spark.mesos.secret is set")
-    }
-    conf.getOption("spark.mesos.role").foreach { role =>
-      fwInfoBuilder.setRole(role)
-    }
-    val maxGpus = conf.getInt("spark.mesos.gpus.max", 0)
-    if (maxGpus > 0) {
-      fwInfoBuilder.addCapabilities(Capability.newBuilder().setType(Capability.Type.GPU_RESOURCES))
-    }
-    if (credBuilder.hasPrincipal) {
-      new MesosSchedulerDriver(
-        scheduler, fwInfoBuilder.build(), masterUrl, credBuilder.build())
-    } else {
-      new MesosSchedulerDriver(scheduler, fwInfoBuilder.build(), masterUrl)
-    }
-  }
-
-  /**
-   * Starts the MesosSchedulerDriver and stores the current running driver to this new instance.
-   * This driver is expected to not be running.
-   * This method returns only after the scheduler has registered with Mesos.
-   */
-  def startScheduler(newDriver: SchedulerDriver): Unit = {
-    synchronized {
-      if (mesosDriver != null) {
-        registerLatch.await()
-        return
-      }
-      @volatile
-      var error: Option[Exception] = None
-
-      // We create a new thread that will block inside `mesosDriver.run`
-      // until the scheduler exists
-      new Thread(Utils.getFormattedClassName(this) + "-mesos-driver") {
-        setDaemon(true)
-        override def run() {
-          try {
-            mesosDriver = newDriver
-            val ret = mesosDriver.run()
-            logInfo("driver.run() returned with code " + ret)
-            if (ret != null && ret.equals(Status.DRIVER_ABORTED)) {
-              error = Some(new SparkException("Error starting driver, DRIVER_ABORTED"))
-              markErr()
-            }
-          } catch {
-            case e: Exception =>
-              logError("driver.run() failed", e)
-              error = Some(e)
-              markErr()
-          }
-        }
-      }.start()
-
-      registerLatch.await()
-
-      // propagate any error to the calling thread. This ensures that SparkContext creation fails
-      // without leaving a broken context that won't be able to schedule any tasks
-      error.foreach(throw _)
-    }
-  }
-
-  def getResource(res: JList[Resource], name: String): Double = {
-    // A resource can have multiple values in the offer since it can either be from
-    // a specific role or wildcard.
-    res.asScala.filter(_.getName == name).map(_.getScalar.getValue).sum
-  }
-
-  /**
-   * Transforms a range resource to a list of ranges
-   *
-   * @param res the mesos resource list
-   * @param name the name of the resource
-   * @return the list of ranges returned
-   */
-  protected def getRangeResource(res: JList[Resource], name: String): List[(Long, Long)] = {
-    // A resource can have multiple values in the offer since it can either be from
-    // a specific role or wildcard.
-    res.asScala.filter(_.getName == name).flatMap(_.getRanges.getRangeList.asScala
-      .map(r => (r.getBegin, r.getEnd)).toList).toList
-  }
-
-  /**
-   * Signal that the scheduler has registered with Mesos.
-   */
-  protected def markRegistered(): Unit = {
-    registerLatch.countDown()
-  }
-
-  protected def markErr(): Unit = {
-    registerLatch.countDown()
-  }
-
-  def createResource(name: String, amount: Double, role: Option[String] = None): Resource = {
-    val builder = Resource.newBuilder()
-      .setName(name)
-      .setType(Value.Type.SCALAR)
-      .setScalar(Value.Scalar.newBuilder().setValue(amount).build())
-
-    role.foreach { r => builder.setRole(r) }
-
-    builder.build()
-  }
-
-  /**
-   * Partition the existing set of resources into two groups, those remaining to be
-   * scheduled and those requested to be used for a new task.
-   *
-   * @param resources The full list of available resources
-   * @param resourceName The name of the resource to take from the available resources
-   * @param amountToUse The amount of resources to take from the available resources
-   * @return The remaining resources list and the used resources list.
-   */
-  def partitionResources(
-      resources: JList[Resource],
-      resourceName: String,
-      amountToUse: Double): (List[Resource], List[Resource]) = {
-    var remain = amountToUse
-    var requestedResources = new ArrayBuffer[Resource]
-    val remainingResources = resources.asScala.map {
-      case r =>
-        if (remain > 0 &&
-          r.getType == Value.Type.SCALAR &&
-          r.getScalar.getValue > 0.0 &&
-          r.getName == resourceName) {
-          val usage = Math.min(remain, r.getScalar.getValue)
-          requestedResources += createResource(resourceName, usage, Some(r.getRole))
-          remain -= usage
-          createResource(resourceName, r.getScalar.getValue - usage, Some(r.getRole))
-        } else {
-          r
-        }
-    }
-
-    // Filter any resource that has depleted.
-    val filteredResources =
-      remainingResources.filter(r => r.getType != Value.Type.SCALAR || r.getScalar.getValue > 0.0)
-
-    (filteredResources.toList, requestedResources.toList)
-  }
-
-  /** Helper method to get the key,value-set pair for a Mesos Attribute protobuf */
-  protected def getAttribute(attr: Attribute): (String, Set[String]) = {
-    (attr.getName, attr.getText.getValue.split(',').toSet)
-  }
-
-
-  /** Build a Mesos resource protobuf object */
-  protected def createResource(resourceName: String, quantity: Double): Protos.Resource = {
-    Resource.newBuilder()
-      .setName(resourceName)
-      .setType(Value.Type.SCALAR)
-      .setScalar(Value.Scalar.newBuilder().setValue(quantity).build())
-      .build()
-  }
-
-  /**
-   * Converts the attributes from the resource offer into a Map of name -> Attribute Value
-   * The attribute values are the mesos attribute types and they are
-   *
-   * @param offerAttributes the attributes offered
-   * @return
-   */
-  protected def toAttributeMap(offerAttributes: JList[Attribute]): Map[String, GeneratedMessage] = {
-    offerAttributes.asScala.map { attr =>
-      val attrValue = attr.getType match {
-        case Value.Type.SCALAR => attr.getScalar
-        case Value.Type.RANGES => attr.getRanges
-        case Value.Type.SET => attr.getSet
-        case Value.Type.TEXT => attr.getText
-      }
-      (attr.getName, attrValue)
-    }.toMap
-  }
-
-
-  /**
-   * Match the requirements (if any) to the offer attributes.
-   * if attribute requirements are not specified - return true
-   * else if attribute is defined and no values are given, simple attribute presence is performed
-   * else if attribute name and value is specified, subset match is performed on slave attributes
-   */
-  def matchesAttributeRequirements(
-      slaveOfferConstraints: Map[String, Set[String]],
-      offerAttributes: Map[String, GeneratedMessage]): Boolean = {
-    slaveOfferConstraints.forall {
-      // offer has the required attribute and subsumes the required values for that attribute
-      case (name, requiredValues) =>
-        offerAttributes.get(name) match {
-          case None => false
-          case Some(_) if requiredValues.isEmpty => true // empty value matches presence
-          case Some(scalarValue: Value.Scalar) =>
-            // check if provided values is less than equal to the offered values
-            requiredValues.map(_.toDouble).exists(_ <= scalarValue.getValue)
-          case Some(rangeValue: Value.Range) =>
-            val offerRange = rangeValue.getBegin to rangeValue.getEnd
-            // Check if there is some required value that is between the ranges specified
-            // Note: We only support the ability to specify discrete values, in the future
-            // we may expand it to subsume ranges specified with a XX..YY value or something
-            // similar to that.
-            requiredValues.map(_.toLong).exists(offerRange.contains(_))
-          case Some(offeredValue: Value.Set) =>
-            // check if the specified required values is a subset of offered set
-            requiredValues.subsetOf(offeredValue.getItemList.asScala.toSet)
-          case Some(textValue: Value.Text) =>
-            // check if the specified value is equal, if multiple values are specified
-            // we succeed if any of them match.
-            requiredValues.contains(textValue.getValue)
-        }
-    }
-  }
-
-  /**
-   * Parses the attributes constraints provided to spark and build a matching data struct:
-   *  Map[<attribute-name>, Set[values-to-match]]
-   *  The constraints are specified as ';' separated key-value pairs where keys and values
-   *  are separated by ':'. The ':' implies equality (for singular values) and "is one of" for
-   *  multiple values (comma separated). For example:
-   *  {{{
-   *  parseConstraintString("os:centos7;zone:us-east-1a,us-east-1b")
-   *  // would result in
-   *  <code>
-   *  Map(
-   *    "os" -> Set("centos7"),
-   *    "zone":   -> Set("us-east-1a", "us-east-1b")
-   *  )
-   *  }}}
-   *
-   *  Mesos documentation: http://mesos.apache.org/documentation/attributes-resources/
-   *                       https://github.com/apache/mesos/blob/master/src/common/values.cpp
-   *                       https://github.com/apache/mesos/blob/master/src/common/attributes.cpp
-   *
-   * @param constraintsVal constaints string consisting of ';' separated key-value pairs (separated
-   *                       by ':')
-   * @return  Map of constraints to match resources offers.
-   */
-  def parseConstraintString(constraintsVal: String): Map[String, Set[String]] = {
-    /*
-      Based on mesos docs:
-      attributes : attribute ( ";" attribute )*
-      attribute : labelString ":" ( labelString | "," )+
-      labelString : [a-zA-Z0-9_/.-]
-    */
-    val splitter = Splitter.on(';').trimResults().withKeyValueSeparator(':')
-    // kv splitter
-    if (constraintsVal.isEmpty) {
-      Map()
-    } else {
-      try {
-        splitter.split(constraintsVal).asScala.toMap.mapValues(v =>
-          if (v == null || v.isEmpty) {
-            Set[String]()
-          } else {
-            v.split(',').toSet
-          }
-        )
-      } catch {
-        case NonFatal(e) =>
-          throw new IllegalArgumentException(s"Bad constraint string: $constraintsVal", e)
-      }
-    }
-  }
-
-  // These defaults copied from YARN
-  private val MEMORY_OVERHEAD_FRACTION = 0.10
-  private val MEMORY_OVERHEAD_MINIMUM = 384
-
-  /**
-   * Return the amount of memory to allocate to each executor, taking into account
-   * container overheads.
-   *
-   * @param sc SparkContext to use to get `spark.mesos.executor.memoryOverhead` value
-   * @return memory requirement as (0.1 * <memoryOverhead>) or MEMORY_OVERHEAD_MINIMUM
-   *         (whichever is larger)
-   */
-  def executorMemory(sc: SparkContext): Int = {
-    sc.conf.getInt("spark.mesos.executor.memoryOverhead",
-      math.max(MEMORY_OVERHEAD_FRACTION * sc.executorMemory, MEMORY_OVERHEAD_MINIMUM).toInt) +
-      sc.executorMemory
-  }
-
-  def setupUris(uris: String,
-                builder: CommandInfo.Builder,
-                useFetcherCache: Boolean = false): Unit = {
-    uris.split(",").foreach { uri =>
-      builder.addUris(CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetcherCache))
-    }
-  }
-
-  protected def getRejectOfferDurationForUnmetConstraints(sc: SparkContext): Long = {
-    sc.conf.getTimeAsSeconds("spark.mesos.rejectOfferDurationForUnmetConstraints", "120s")
-  }
-
-  protected def getRejectOfferDurationForReachedMaxCores(sc: SparkContext): Long = {
-    sc.conf.getTimeAsSeconds("spark.mesos.rejectOfferDurationForReachedMaxCores", "120s")
-  }
-
-  /**
-   * Checks executor ports if they are within some range of the offered list of ports ranges,
-   *
-   * @param conf the Spark Config
-   * @param ports the list of ports to check
-   * @return true if ports are within range false otherwise
-   */
-  protected def checkPorts(conf: SparkConf, ports: List[(Long, Long)]): Boolean = {
-
-    def checkIfInRange(port: Long, ps: List[(Long, Long)]): Boolean = {
-      ps.exists{case (rangeStart, rangeEnd) => rangeStart <= port & rangeEnd >= port }
-    }
-
-    val portsToCheck = nonZeroPortValuesFromConfig(conf)
-    val withinRange = portsToCheck.forall(p => checkIfInRange(p, ports))
-    // make sure we have enough ports to allocate per offer
-    val enoughPorts =
-    ports.map{case (rangeStart, rangeEnd) => rangeEnd - rangeStart + 1}.sum >= portsToCheck.size
-    enoughPorts && withinRange
-  }
-
-  /**
-   * Partitions port resources.
-   *
-   * @param requestedPorts non-zero ports to assign
-   * @param offeredResources the resources offered
-   * @return resources left, port resources to be used.
-   */
-  def partitionPortResources(requestedPorts: List[Long], offeredResources: List[Resource])
-    : (List[Resource], List[Resource]) = {
-    if (requestedPorts.isEmpty) {
-      (offeredResources, List[Resource]())
-    } else {
-      // partition port offers
-      val (resourcesWithoutPorts, portResources) = filterPortResources(offeredResources)
-
-      val portsAndRoles = requestedPorts.
-        map(x => (x, findPortAndGetAssignedRangeRole(x, portResources)))
-
-      val assignedPortResources = createResourcesFromPorts(portsAndRoles)
-
-      // ignore non-assigned port resources, they will be declined implicitly by mesos
-      // no need for splitting port resources.
-      (resourcesWithoutPorts, assignedPortResources)
-    }
-  }
-
-  val managedPortNames = List("spark.executor.port", BLOCK_MANAGER_PORT.key)
-
-  /**
-   * The values of the non-zero ports to be used by the executor process.
-   * @param conf the spark config to use
-   * @return the ono-zero values of the ports
-   */
-  def nonZeroPortValuesFromConfig(conf: SparkConf): List[Long] = {
-    managedPortNames.map(conf.getLong(_, 0)).filter( _ != 0)
-  }
-
-  /** Creates a mesos resource for a specific port number. */
-  private def createResourcesFromPorts(portsAndRoles: List[(Long, String)]) : List[Resource] = {
-    portsAndRoles.flatMap{ case (port, role) =>
-      createMesosPortResource(List((port, port)), Some(role))}
-  }
-
-  /** Helper to create mesos resources for specific port ranges. */
-  private def createMesosPortResource(
-      ranges: List[(Long, Long)],
-      role: Option[String] = None): List[Resource] = {
-    ranges.map { case (rangeStart, rangeEnd) =>
-      val rangeValue = Value.Range.newBuilder()
-        .setBegin(rangeStart)
-        .setEnd(rangeEnd)
-      val builder = Resource.newBuilder()
-        .setName("ports")
-        .setType(Value.Type.RANGES)
-        .setRanges(Value.Ranges.newBuilder().addRange(rangeValue))
-      role.foreach(r => builder.setRole(r))
-      builder.build()
-    }
-  }
-
- /**
-  * Helper to assign a port to an offered range and get the latter's role
-  * info to use it later on.
-  */
-  private def findPortAndGetAssignedRangeRole(port: Long, portResources: List[Resource])
-    : String = {
-
-    val ranges = portResources.
-      map(resource =>
-        (resource.getRole, resource.getRanges.getRangeList.asScala
-          .map(r => (r.getBegin, r.getEnd)).toList))
-
-    val rangePortRole = ranges
-      .find { case (role, rangeList) => rangeList
-        .exists{ case (rangeStart, rangeEnd) => rangeStart <= port & rangeEnd >= port}}
-    // this is safe since we have previously checked about the ranges (see checkPorts method)
-    rangePortRole.map{ case (role, rangeList) => role}.get
-  }
-
-  /** Retrieves the port resources from a list of mesos offered resources */
-  private def filterPortResources(resources: List[Resource]): (List[Resource], List[Resource]) = {
-    resources.partition { r => !(r.getType == Value.Type.RANGES && r.getName == "ports") }
-  }
-
-  /**
-   * spark.mesos.driver.frameworkId is set by the cluster dispatcher to correlate driver
-   * submissions with frameworkIDs.  However, this causes issues when a driver process launches
-   * more than one framework (more than one SparkContext(, because they all try to register with
-   * the same frameworkID.  To enforce that only the first driver registers with the configured
-   * framework ID, the driver calls this method after the first registration.
-   */
-  def unsetFrameworkID(sc: SparkContext) {
-    sc.conf.remove("spark.mesos.driver.frameworkId")
-    System.clearProperty("spark.mesos.driver.frameworkId")
-  }
-
-  def mesosToTaskState(state: MesosTaskState): TaskState.TaskState = state match {
-    case MesosTaskState.TASK_STAGING | MesosTaskState.TASK_STARTING => TaskState.LAUNCHING
-    case MesosTaskState.TASK_RUNNING | MesosTaskState.TASK_KILLING => TaskState.RUNNING
-    case MesosTaskState.TASK_FINISHED => TaskState.FINISHED
-    case MesosTaskState.TASK_FAILED => TaskState.FAILED
-    case MesosTaskState.TASK_KILLED => TaskState.KILLED
-    case MesosTaskState.TASK_LOST | MesosTaskState.TASK_ERROR => TaskState.LOST
-  }
-
-  def taskStateToMesos(state: TaskState.TaskState): MesosTaskState = state match {
-    case TaskState.LAUNCHING => MesosTaskState.TASK_STARTING
-    case TaskState.RUNNING => MesosTaskState.TASK_RUNNING
-    case TaskState.FINISHED => MesosTaskState.TASK_FINISHED
-    case TaskState.FAILED => MesosTaskState.TASK_FAILED
-    case TaskState.KILLED => MesosTaskState.TASK_KILLED
-    case TaskState.LOST => MesosTaskState.TASK_LOST
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala
----------------------------------------------------------------------
diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala
deleted file mode 100644
index 8370b61..0000000
--- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala
+++ /dev/null
@@ -1,51 +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.mesos
-
-import java.nio.ByteBuffer
-
-import org.apache.mesos.protobuf.ByteString
-
-import org.apache.spark.internal.Logging
-
-/**
- * Wrapper for serializing the data sent when launching Mesos tasks.
- */
-private[spark] case class MesosTaskLaunchData(
-  serializedTask: ByteBuffer,
-  attemptNumber: Int) extends Logging {
-
-  def toByteString: ByteString = {
-    val dataBuffer = ByteBuffer.allocate(4 + serializedTask.limit)
-    dataBuffer.putInt(attemptNumber)
-    dataBuffer.put(serializedTask)
-    dataBuffer.rewind
-    logDebug(s"ByteBuffer size: [${dataBuffer.remaining}]")
-    ByteString.copyFrom(dataBuffer)
-  }
-}
-
-private[spark] object MesosTaskLaunchData extends Logging {
-  def fromByteString(byteString: ByteString): MesosTaskLaunchData = {
-    val byteBuffer = byteString.asReadOnlyByteBuffer()
-    logDebug(s"ByteBuffer size: [${byteBuffer.remaining}]")
-    val attemptNumber = byteBuffer.getInt // updates the position by 4 bytes
-    val serializedTask = byteBuffer.slice() // subsequence starting at the current position
-    MesosTaskLaunchData(serializedTask, attemptNumber)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala
----------------------------------------------------------------------
diff --git a/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala b/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala
deleted file mode 100644
index 33e7d69..0000000
--- a/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala
+++ /dev/null
@@ -1,63 +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.deploy.mesos
-
-import org.apache.spark.{SparkConf, SparkFunSuite}
-import org.apache.spark.deploy.TestPrematureExit
-
-class MesosClusterDispatcherArgumentsSuite extends SparkFunSuite
-  with TestPrematureExit {
-
-  test("test if spark config args are passed sucessfully") {
-    val args = Array[String]("--master", "mesos://localhost:5050", "--conf", "key1=value1",
-      "--conf", "spark.mesos.key2=value2", "--verbose")
-    val conf = new SparkConf()
-    new MesosClusterDispatcherArguments(args, conf)
-
-    assert(conf.getOption("key1").isEmpty)
-    assert(conf.get("spark.mesos.key2") == "value2")
-  }
-
-  test("test non conf settings") {
-    val masterUrl = "mesos://localhost:5050"
-    val port = "1212"
-    val zookeeperUrl = "zk://localhost:2181"
-    val host = "localhost"
-    val webUiPort = "2323"
-    val name = "myFramework"
-
-    val args1 = Array("--master", masterUrl, "--verbose", "--name", name)
-    val args2 = Array("-p", port, "-h", host, "-z", zookeeperUrl)
-    val args3 = Array("--webui-port", webUiPort)
-
-    val args = args1 ++ args2 ++ args3
-    val conf = new SparkConf()
-    val mesosDispClusterArgs = new MesosClusterDispatcherArguments(args, conf)
-
-    assert(mesosDispClusterArgs.verbose)
-    assert(mesosDispClusterArgs.confProperties.isEmpty)
-    assert(mesosDispClusterArgs.host == host)
-    assert(Option(mesosDispClusterArgs.masterUrl).isDefined)
-    assert(mesosDispClusterArgs.masterUrl == masterUrl.stripPrefix("mesos://"))
-    assert(Option(mesosDispClusterArgs.zookeeperUrl).isDefined)
-    assert(mesosDispClusterArgs.zookeeperUrl == Some(zookeeperUrl))
-    assert(mesosDispClusterArgs.name == name)
-    assert(mesosDispClusterArgs.webUiPort == webUiPort.toInt)
-    assert(mesosDispClusterArgs.port == port.toInt)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherSuite.scala
----------------------------------------------------------------------
diff --git a/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherSuite.scala b/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherSuite.scala
deleted file mode 100644
index 7484e3b..0000000
--- a/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherSuite.scala
+++ /dev/null
@@ -1,40 +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.deploy.mesos
-
-import org.apache.spark.SparkFunSuite
-import org.apache.spark.deploy.TestPrematureExit
-
-class MesosClusterDispatcherSuite extends SparkFunSuite
-  with TestPrematureExit{
-
-  test("prints usage on empty input") {
-    testPrematureExit(Array[String](),
-      "Usage: MesosClusterDispatcher", MesosClusterDispatcher)
-  }
-
-  test("prints usage with only --help") {
-    testPrematureExit(Array("--help"),
-      "Usage: MesosClusterDispatcher", MesosClusterDispatcher)
-  }
-
-  test("prints error with unrecognized options") {
-    testPrematureExit(Array("--blarg"), "Unrecognized option: '--blarg'", MesosClusterDispatcher)
-    testPrematureExit(Array("-bleg"), "Unrecognized option: '-bleg'", MesosClusterDispatcher)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala
----------------------------------------------------------------------
diff --git a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala
deleted file mode 100644
index a558554..0000000
--- a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala
+++ /dev/null
@@ -1,56 +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.mesos
-
-import org.apache.spark._
-import org.apache.spark.internal.config._
-
-class MesosClusterManagerSuite extends SparkFunSuite with LocalSparkContext {
-    def testURL(masterURL: String, expectedClass: Class[_], coarse: Boolean) {
-      val conf = new SparkConf().set("spark.mesos.coarse", coarse.toString)
-      sc = new SparkContext("local", "test", conf)
-      val clusterManager = new MesosClusterManager()
-
-      assert(clusterManager.canCreate(masterURL))
-      val taskScheduler = clusterManager.createTaskScheduler(sc, masterURL)
-      val sched = clusterManager.createSchedulerBackend(sc, masterURL, taskScheduler)
-      assert(sched.getClass === expectedClass)
-    }
-
-    test("mesos fine-grained") {
-      testURL("mesos://localhost:1234", classOf[MesosFineGrainedSchedulerBackend], coarse = false)
-    }
-
-    test("mesos coarse-grained") {
-      testURL("mesos://localhost:1234", classOf[MesosCoarseGrainedSchedulerBackend], coarse = true)
-    }
-
-    test("mesos with zookeeper") {
-      testURL("mesos://zk://localhost:1234,localhost:2345",
-          classOf[MesosFineGrainedSchedulerBackend],
-          coarse = false)
-    }
-
-    test("mesos with i/o encryption throws error") {
-      val se = intercept[SparkException] {
-        val conf = new SparkConf().setAppName("test").set(IO_ENCRYPTION_ENABLED, true)
-        sc = new SparkContext("mesos", "test", conf)
-      }
-      assert(se.getCause().isInstanceOf[IllegalArgumentException])
-    }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala
----------------------------------------------------------------------
diff --git a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala
deleted file mode 100644
index 74e5ce2..0000000
--- a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala
+++ /dev/null
@@ -1,239 +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.mesos
-
-import java.util.{Collection, Collections, Date}
-
-import scala.collection.JavaConverters._
-
-import org.apache.mesos.Protos._
-import org.apache.mesos.Protos.Value.{Scalar, Type}
-import org.apache.mesos.SchedulerDriver
-import org.mockito.{ArgumentCaptor, Matchers}
-import org.mockito.Mockito._
-import org.scalatest.mock.MockitoSugar
-
-import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite}
-import org.apache.spark.deploy.Command
-import org.apache.spark.deploy.mesos.MesosDriverDescription
-
-class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar {
-
-  private val command = new Command("mainClass", Seq("arg"), Map(), Seq(), Seq(), Seq())
-  private var driver: SchedulerDriver = _
-  private var scheduler: MesosClusterScheduler = _
-
-  private def setScheduler(sparkConfVars: Map[String, String] = null): Unit = {
-    val conf = new SparkConf()
-    conf.setMaster("mesos://localhost:5050")
-    conf.setAppName("spark mesos")
-
-    if (sparkConfVars != null) {
-      conf.setAll(sparkConfVars)
-    }
-
-    driver = mock[SchedulerDriver]
-    scheduler = new MesosClusterScheduler(
-      new BlackHoleMesosClusterPersistenceEngineFactory, conf) {
-      override def start(): Unit = { ready = true }
-    }
-    scheduler.start()
-  }
-
-  test("can queue drivers") {
-    setScheduler()
-
-    val response = scheduler.submitDriver(
-      new MesosDriverDescription("d1", "jar", 1000, 1, true,
-        command, Map[String, String](), "s1", new Date()))
-    assert(response.success)
-    val response2 =
-      scheduler.submitDriver(new MesosDriverDescription(
-        "d1", "jar", 1000, 1, true, command, Map[String, String](), "s2", new Date()))
-    assert(response2.success)
-    val state = scheduler.getSchedulerState()
-    val queuedDrivers = state.queuedDrivers.toList
-    assert(queuedDrivers(0).submissionId == response.submissionId)
-    assert(queuedDrivers(1).submissionId == response2.submissionId)
-  }
-
-  test("can kill queued drivers") {
-    setScheduler()
-
-    val response = scheduler.submitDriver(
-        new MesosDriverDescription("d1", "jar", 1000, 1, true,
-          command, Map[String, String](), "s1", new Date()))
-    assert(response.success)
-    val killResponse = scheduler.killDriver(response.submissionId)
-    assert(killResponse.success)
-    val state = scheduler.getSchedulerState()
-    assert(state.queuedDrivers.isEmpty)
-  }
-
-  test("can handle multiple roles") {
-    setScheduler()
-
-    val driver = mock[SchedulerDriver]
-    val response = scheduler.submitDriver(
-      new MesosDriverDescription("d1", "jar", 1200, 1.5, true,
-        command,
-        Map(("spark.mesos.executor.home", "test"), ("spark.app.name", "test")),
-        "s1",
-        new Date()))
-    assert(response.success)
-    val offer = Offer.newBuilder()
-      .addResources(
-        Resource.newBuilder().setRole("*")
-          .setScalar(Scalar.newBuilder().setValue(1).build()).setName("cpus").setType(Type.SCALAR))
-      .addResources(
-        Resource.newBuilder().setRole("*")
-          .setScalar(Scalar.newBuilder().setValue(1000).build())
-          .setName("mem")
-          .setType(Type.SCALAR))
-      .addResources(
-        Resource.newBuilder().setRole("role2")
-          .setScalar(Scalar.newBuilder().setValue(1).build()).setName("cpus").setType(Type.SCALAR))
-      .addResources(
-        Resource.newBuilder().setRole("role2")
-          .setScalar(Scalar.newBuilder().setValue(500).build()).setName("mem").setType(Type.SCALAR))
-      .setId(OfferID.newBuilder().setValue("o1").build())
-      .setFrameworkId(FrameworkID.newBuilder().setValue("f1").build())
-      .setSlaveId(SlaveID.newBuilder().setValue("s1").build())
-      .setHostname("host1")
-      .build()
-
-    val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]])
-
-    when(
-      driver.launchTasks(
-        Matchers.eq(Collections.singleton(offer.getId)),
-        capture.capture())
-    ).thenReturn(Status.valueOf(1))
-
-    scheduler.resourceOffers(driver, Collections.singletonList(offer))
-
-    val taskInfos = capture.getValue
-    assert(taskInfos.size() == 1)
-    val taskInfo = taskInfos.iterator().next()
-    val resources = taskInfo.getResourcesList
-    assert(scheduler.getResource(resources, "cpus") == 1.5)
-    assert(scheduler.getResource(resources, "mem") == 1200)
-    val resourcesSeq: Seq[Resource] = resources.asScala
-    val cpus = resourcesSeq.filter(_.getName.equals("cpus")).toList
-    assert(cpus.size == 2)
-    assert(cpus.exists(_.getRole().equals("role2")))
-    assert(cpus.exists(_.getRole().equals("*")))
-    val mem = resourcesSeq.filter(_.getName.equals("mem")).toList
-    assert(mem.size == 2)
-    assert(mem.exists(_.getRole().equals("role2")))
-    assert(mem.exists(_.getRole().equals("*")))
-
-    verify(driver, times(1)).launchTasks(
-      Matchers.eq(Collections.singleton(offer.getId)),
-      capture.capture()
-    )
-  }
-
-  test("escapes commandline args for the shell") {
-    setScheduler()
-
-    val conf = new SparkConf()
-    conf.setMaster("mesos://localhost:5050")
-    conf.setAppName("spark mesos")
-    val scheduler = new MesosClusterScheduler(
-      new BlackHoleMesosClusterPersistenceEngineFactory, conf) {
-      override def start(): Unit = { ready = true }
-    }
-    val escape = scheduler.shellEscape _
-    def wrapped(str: String): String = "\"" + str + "\""
-
-    // Wrapped in quotes
-    assert(escape("'should be left untouched'") === "'should be left untouched'")
-    assert(escape("\"should be left untouched\"") === "\"should be left untouched\"")
-
-    // Harmless
-    assert(escape("") === "")
-    assert(escape("harmless") === "harmless")
-    assert(escape("har-m.l3ss") === "har-m.l3ss")
-
-    // Special Chars escape
-    assert(escape("should escape this \" quote") === wrapped("should escape this \\\" quote"))
-    assert(escape("shouldescape\"quote") === wrapped("shouldescape\\\"quote"))
-    assert(escape("should escape this $ dollar") === wrapped("should escape this \\$ dollar"))
-    assert(escape("should escape this ` backtick") === wrapped("should escape this \\` backtick"))
-    assert(escape("""should escape this \ backslash""")
-      === wrapped("""should escape this \\ backslash"""))
-    assert(escape("""\"?""") === wrapped("""\\\"?"""))
-
-
-    // Special Chars no escape only wrap
-    List(" ", "'", "<", ">", "&", "|", "?", "*", ";", "!", "#", "(", ")").foreach(char => {
-      assert(escape(s"onlywrap${char}this") === wrapped(s"onlywrap${char}this"))
-    })
-  }
-
-  test("supports spark.mesos.driverEnv.*") {
-    setScheduler()
-
-    val mem = 1000
-    val cpu = 1
-
-    val response = scheduler.submitDriver(
-      new MesosDriverDescription("d1", "jar", mem, cpu, true,
-        command,
-        Map("spark.mesos.executor.home" -> "test",
-          "spark.app.name" -> "test",
-          "spark.mesos.driverEnv.TEST_ENV" -> "TEST_VAL"),
-        "s1",
-        new Date()))
-    assert(response.success)
-
-    val offer = Utils.createOffer("o1", "s1", mem, cpu)
-    scheduler.resourceOffers(driver, List(offer).asJava)
-    val tasks = Utils.verifyTaskLaunched(driver, "o1")
-    val env = tasks.head.getCommand.getEnvironment.getVariablesList.asScala.map(v =>
-      (v.getName, v.getValue)).toMap
-    assert(env.getOrElse("TEST_ENV", null) == "TEST_VAL")
-  }
-
-  test("supports spark.mesos.network.name") {
-    setScheduler()
-
-    val mem = 1000
-    val cpu = 1
-
-    val response = scheduler.submitDriver(
-      new MesosDriverDescription("d1", "jar", mem, cpu, true,
-        command,
-        Map("spark.mesos.executor.home" -> "test",
-          "spark.app.name" -> "test",
-          "spark.mesos.network.name" -> "test-network-name"),
-        "s1",
-        new Date()))
-
-    assert(response.success)
-
-    val offer = Utils.createOffer("o1", "s1", mem, cpu)
-    scheduler.resourceOffers(driver, List(offer).asJava)
-
-    val launchedTasks = Utils.verifyTaskLaunched(driver, "o1")
-    val networkInfos = launchedTasks.head.getContainer.getNetworkInfosList
-    assert(networkInfos.size == 1)
-    assert(networkInfos.get(0).getName == "test-network-name")
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala
----------------------------------------------------------------------
diff --git a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala
deleted file mode 100644
index a674da4..0000000
--- a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala
+++ /dev/null
@@ -1,601 +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.mesos
-
-import java.util.concurrent.TimeUnit
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-import scala.concurrent.duration._
-import scala.concurrent.Promise
-import scala.reflect.ClassTag
-
-import org.apache.mesos.{Protos, Scheduler, SchedulerDriver}
-import org.apache.mesos.Protos._
-import org.mockito.Matchers
-import org.mockito.Matchers._
-import org.mockito.Mockito._
-import org.scalatest.concurrent.ScalaFutures
-import org.scalatest.mock.MockitoSugar
-import org.scalatest.BeforeAndAfter
-
-import org.apache.spark.{LocalSparkContext, SecurityManager, SparkConf, SparkContext, SparkFunSuite}
-import org.apache.spark.internal.config._
-import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient
-import org.apache.spark.rpc.RpcEndpointRef
-import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor
-import org.apache.spark.scheduler.TaskSchedulerImpl
-import org.apache.spark.scheduler.cluster.mesos.Utils._
-
-class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite
-    with LocalSparkContext
-    with MockitoSugar
-    with BeforeAndAfter
-    with ScalaFutures {
-
-  private var sparkConf: SparkConf = _
-  private var driver: SchedulerDriver = _
-  private var taskScheduler: TaskSchedulerImpl = _
-  private var backend: MesosCoarseGrainedSchedulerBackend = _
-  private var externalShuffleClient: MesosExternalShuffleClient = _
-  private var driverEndpoint: RpcEndpointRef = _
-  @volatile private var stopCalled = false
-
-  // All 'requests' to the scheduler run immediately on the same thread, so
-  // demand that all futures have their value available immediately.
-  implicit override val patienceConfig = PatienceConfig(timeout = Duration(0, TimeUnit.SECONDS))
-
-  test("mesos supports killing and limiting executors") {
-    setBackend()
-    sparkConf.set("spark.driver.host", "driverHost")
-    sparkConf.set("spark.driver.port", "1234")
-
-    val minMem = backend.executorMemory(sc)
-    val minCpu = 4
-    val offers = List(Resources(minMem, minCpu))
-
-    // launches a task on a valid offer
-    offerResources(offers)
-    verifyTaskLaunched(driver, "o1")
-
-    // kills executors
-    assert(backend.doRequestTotalExecutors(0).futureValue)
-    assert(backend.doKillExecutors(Seq("0")).futureValue)
-    val taskID0 = createTaskId("0")
-    verify(driver, times(1)).killTask(taskID0)
-
-    // doesn't launch a new task when requested executors == 0
-    offerResources(offers, 2)
-    verifyDeclinedOffer(driver, createOfferId("o2"))
-
-    // Launches a new task when requested executors is positive
-    backend.doRequestTotalExecutors(2)
-    offerResources(offers, 2)
-    verifyTaskLaunched(driver, "o2")
-  }
-
-  test("mesos supports killing and relaunching tasks with executors") {
-    setBackend()
-
-    // launches a task on a valid offer
-    val minMem = backend.executorMemory(sc) + 1024
-    val minCpu = 4
-    val offer1 = Resources(minMem, minCpu)
-    val offer2 = Resources(minMem, 1)
-    offerResources(List(offer1, offer2))
-    verifyTaskLaunched(driver, "o1")
-
-    // accounts for a killed task
-    val status = createTaskStatus("0", "s1", TaskState.TASK_KILLED)
-    backend.statusUpdate(driver, status)
-    verify(driver, times(1)).reviveOffers()
-
-    // Launches a new task on a valid offer from the same slave
-    offerResources(List(offer2))
-    verifyTaskLaunched(driver, "o2")
-  }
-
-  test("mesos supports spark.executor.cores") {
-    val executorCores = 4
-    setBackend(Map("spark.executor.cores" -> executorCores.toString))
-
-    val executorMemory = backend.executorMemory(sc)
-    val offers = List(Resources(executorMemory * 2, executorCores + 1))
-    offerResources(offers)
-
-    val taskInfos = verifyTaskLaunched(driver, "o1")
-    assert(taskInfos.length == 1)
-
-    val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus")
-    assert(cpus == executorCores)
-  }
-
-  test("mesos supports unset spark.executor.cores") {
-    setBackend()
-
-    val executorMemory = backend.executorMemory(sc)
-    val offerCores = 10
-    offerResources(List(Resources(executorMemory * 2, offerCores)))
-
-    val taskInfos = verifyTaskLaunched(driver, "o1")
-    assert(taskInfos.length == 1)
-
-    val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus")
-    assert(cpus == offerCores)
-  }
-
-  test("mesos does not acquire more than spark.cores.max") {
-    val maxCores = 10
-    setBackend(Map("spark.cores.max" -> maxCores.toString))
-
-    val executorMemory = backend.executorMemory(sc)
-    offerResources(List(Resources(executorMemory, maxCores + 1)))
-
-    val taskInfos = verifyTaskLaunched(driver, "o1")
-    assert(taskInfos.length == 1)
-
-    val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus")
-    assert(cpus == maxCores)
-  }
-
-  test("mesos does not acquire gpus if not specified") {
-    setBackend()
-
-    val executorMemory = backend.executorMemory(sc)
-    offerResources(List(Resources(executorMemory, 1, 1)))
-
-    val taskInfos = verifyTaskLaunched(driver, "o1")
-    assert(taskInfos.length == 1)
-
-    val gpus = backend.getResource(taskInfos.head.getResourcesList, "gpus")
-    assert(gpus == 0.0)
-  }
-
-
-  test("mesos does not acquire more than spark.mesos.gpus.max") {
-    val maxGpus = 5
-    setBackend(Map("spark.mesos.gpus.max" -> maxGpus.toString))
-
-    val executorMemory = backend.executorMemory(sc)
-    offerResources(List(Resources(executorMemory, 1, maxGpus + 1)))
-
-    val taskInfos = verifyTaskLaunched(driver, "o1")
-    assert(taskInfos.length == 1)
-
-    val gpus = backend.getResource(taskInfos.head.getResourcesList, "gpus")
-    assert(gpus == maxGpus)
-  }
-
-
-  test("mesos declines offers that violate attribute constraints") {
-    setBackend(Map("spark.mesos.constraints" -> "x:true"))
-    offerResources(List(Resources(backend.executorMemory(sc), 4)))
-    verifyDeclinedOffer(driver, createOfferId("o1"), true)
-  }
-
-  test("mesos declines offers with a filter when reached spark.cores.max") {
-    val maxCores = 3
-    setBackend(Map("spark.cores.max" -> maxCores.toString))
-
-    val executorMemory = backend.executorMemory(sc)
-    offerResources(List(
-      Resources(executorMemory, maxCores + 1),
-      Resources(executorMemory, maxCores + 1)))
-
-    verifyTaskLaunched(driver, "o1")
-    verifyDeclinedOffer(driver, createOfferId("o2"), true)
-  }
-
-  test("mesos assigns tasks round-robin on offers") {
-    val executorCores = 4
-    val maxCores = executorCores * 2
-    setBackend(Map("spark.executor.cores" -> executorCores.toString,
-      "spark.cores.max" -> maxCores.toString))
-
-    val executorMemory = backend.executorMemory(sc)
-    offerResources(List(
-      Resources(executorMemory * 2, executorCores * 2),
-      Resources(executorMemory * 2, executorCores * 2)))
-
-    verifyTaskLaunched(driver, "o1")
-    verifyTaskLaunched(driver, "o2")
-  }
-
-  test("mesos creates multiple executors on a single slave") {
-    val executorCores = 4
-    setBackend(Map("spark.executor.cores" -> executorCores.toString))
-
-    // offer with room for two executors
-    val executorMemory = backend.executorMemory(sc)
-    offerResources(List(Resources(executorMemory * 2, executorCores * 2)))
-
-    // verify two executors were started on a single offer
-    val taskInfos = verifyTaskLaunched(driver, "o1")
-    assert(taskInfos.length == 2)
-  }
-
-  test("mesos doesn't register twice with the same shuffle service") {
-    setBackend(Map("spark.shuffle.service.enabled" -> "true"))
-    val (mem, cpu) = (backend.executorMemory(sc), 4)
-
-    val offer1 = createOffer("o1", "s1", mem, cpu)
-    backend.resourceOffers(driver, List(offer1).asJava)
-    verifyTaskLaunched(driver, "o1")
-
-    val offer2 = createOffer("o2", "s1", mem, cpu)
-    backend.resourceOffers(driver, List(offer2).asJava)
-    verifyTaskLaunched(driver, "o2")
-
-    val status1 = createTaskStatus("0", "s1", TaskState.TASK_RUNNING)
-    backend.statusUpdate(driver, status1)
-
-    val status2 = createTaskStatus("1", "s1", TaskState.TASK_RUNNING)
-    backend.statusUpdate(driver, status2)
-    verify(externalShuffleClient, times(1))
-      .registerDriverWithShuffleService(anyString, anyInt, anyLong, anyLong)
-  }
-
-  test("Port offer decline when there is no appropriate range") {
-    setBackend(Map(BLOCK_MANAGER_PORT.key -> "30100"))
-    val offeredPorts = (31100L, 31200L)
-    val (mem, cpu) = (backend.executorMemory(sc), 4)
-
-    val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts))
-    backend.resourceOffers(driver, List(offer1).asJava)
-    verify(driver, times(1)).declineOffer(offer1.getId)
-  }
-
-  test("Port offer accepted when ephemeral ports are used") {
-    setBackend()
-    val offeredPorts = (31100L, 31200L)
-    val (mem, cpu) = (backend.executorMemory(sc), 4)
-
-    val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts))
-    backend.resourceOffers(driver, List(offer1).asJava)
-    verifyTaskLaunched(driver, "o1")
-  }
-
-  test("Port offer accepted with user defined port numbers") {
-    val port = 30100
-    setBackend(Map(BLOCK_MANAGER_PORT.key -> s"$port"))
-    val offeredPorts = (30000L, 31000L)
-    val (mem, cpu) = (backend.executorMemory(sc), 4)
-
-    val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts))
-    backend.resourceOffers(driver, List(offer1).asJava)
-    val taskInfo = verifyTaskLaunched(driver, "o1")
-
-    val taskPortResources = taskInfo.head.getResourcesList.asScala.
-    find(r => r.getType == Value.Type.RANGES && r.getName == "ports")
-
-    val isPortInOffer = (r: Resource) => {
-      r.getRanges().getRangeList
-        .asScala.exists(range => range.getBegin == port && range.getEnd == port)
-    }
-    assert(taskPortResources.exists(isPortInOffer))
-  }
-
-  test("mesos kills an executor when told") {
-    setBackend()
-
-    val (mem, cpu) = (backend.executorMemory(sc), 4)
-
-    val offer1 = createOffer("o1", "s1", mem, cpu)
-    backend.resourceOffers(driver, List(offer1).asJava)
-    verifyTaskLaunched(driver, "o1")
-
-    backend.doKillExecutors(List("0"))
-    verify(driver, times(1)).killTask(createTaskId("0"))
-  }
-
-  test("weburi is set in created scheduler driver") {
-    setBackend()
-    val taskScheduler = mock[TaskSchedulerImpl]
-    when(taskScheduler.sc).thenReturn(sc)
-    val driver = mock[SchedulerDriver]
-    when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING)
-    val securityManager = mock[SecurityManager]
-
-    val backend = new MesosCoarseGrainedSchedulerBackend(
-        taskScheduler, sc, "master", securityManager) {
-      override protected def createSchedulerDriver(
-        masterUrl: String,
-        scheduler: Scheduler,
-        sparkUser: String,
-        appName: String,
-        conf: SparkConf,
-        webuiUrl: Option[String] = None,
-        checkpoint: Option[Boolean] = None,
-        failoverTimeout: Option[Double] = None,
-        frameworkId: Option[String] = None): SchedulerDriver = {
-        markRegistered()
-        assert(webuiUrl.isDefined)
-        assert(webuiUrl.get.equals("http://webui"))
-        driver
-      }
-    }
-
-    backend.start()
-  }
-
-  test("honors unset spark.mesos.containerizer") {
-    setBackend(Map("spark.mesos.executor.docker.image" -> "test"))
-
-    val (mem, cpu) = (backend.executorMemory(sc), 4)
-
-    val offer1 = createOffer("o1", "s1", mem, cpu)
-    backend.resourceOffers(driver, List(offer1).asJava)
-
-    val taskInfos = verifyTaskLaunched(driver, "o1")
-    assert(taskInfos.head.getContainer.getType == ContainerInfo.Type.DOCKER)
-  }
-
-  test("honors spark.mesos.containerizer=\"mesos\"") {
-    setBackend(Map(
-      "spark.mesos.executor.docker.image" -> "test",
-      "spark.mesos.containerizer" -> "mesos"))
-
-    val (mem, cpu) = (backend.executorMemory(sc), 4)
-
-    val offer1 = createOffer("o1", "s1", mem, cpu)
-    backend.resourceOffers(driver, List(offer1).asJava)
-
-    val taskInfos = verifyTaskLaunched(driver, "o1")
-    assert(taskInfos.head.getContainer.getType == ContainerInfo.Type.MESOS)
-  }
-
-  test("docker settings are reflected in created tasks") {
-    setBackend(Map(
-      "spark.mesos.executor.docker.image" -> "some_image",
-      "spark.mesos.executor.docker.forcePullImage" -> "true",
-      "spark.mesos.executor.docker.volumes" -> "/host_vol:/container_vol:ro",
-      "spark.mesos.executor.docker.portmaps" -> "8080:80:tcp"
-    ))
-
-    val (mem, cpu) = (backend.executorMemory(sc), 4)
-
-    val offer1 = createOffer("o1", "s1", mem, cpu)
-    backend.resourceOffers(driver, List(offer1).asJava)
-
-    val launchedTasks = verifyTaskLaunched(driver, "o1")
-    assert(launchedTasks.size == 1)
-
-    val containerInfo = launchedTasks.head.getContainer
-    assert(containerInfo.getType == ContainerInfo.Type.DOCKER)
-
-    val volumes = containerInfo.getVolumesList.asScala
-    assert(volumes.size == 1)
-
-    val volume = volumes.head
-    assert(volume.getHostPath == "/host_vol")
-    assert(volume.getContainerPath == "/container_vol")
-    assert(volume.getMode == Volume.Mode.RO)
-
-    val dockerInfo = containerInfo.getDocker
-
-    val portMappings = dockerInfo.getPortMappingsList.asScala
-    assert(portMappings.size == 1)
-
-    val portMapping = portMappings.head
-    assert(portMapping.getHostPort == 8080)
-    assert(portMapping.getContainerPort == 80)
-    assert(portMapping.getProtocol == "tcp")
-  }
-
-  test("force-pull-image option is disabled by default") {
-    setBackend(Map(
-      "spark.mesos.executor.docker.image" -> "some_image"
-    ))
-
-    val (mem, cpu) = (backend.executorMemory(sc), 4)
-
-    val offer1 = createOffer("o1", "s1", mem, cpu)
-    backend.resourceOffers(driver, List(offer1).asJava)
-
-    val launchedTasks = verifyTaskLaunched(driver, "o1")
-    assert(launchedTasks.size == 1)
-
-    val containerInfo = launchedTasks.head.getContainer
-    assert(containerInfo.getType == ContainerInfo.Type.DOCKER)
-
-    val dockerInfo = containerInfo.getDocker
-
-    assert(dockerInfo.getImage == "some_image")
-    assert(!dockerInfo.getForcePullImage)
-  }
-
-  test("Do not call removeExecutor() after backend is stopped") {
-    setBackend()
-
-    // launches a task on a valid offer
-    val offers = List(Resources(backend.executorMemory(sc), 1))
-    offerResources(offers)
-    verifyTaskLaunched(driver, "o1")
-
-    // launches a thread simulating status update
-    val statusUpdateThread = new Thread {
-      override def run(): Unit = {
-        while (!stopCalled) {
-          Thread.sleep(100)
-        }
-
-        val status = createTaskStatus("0", "s1", TaskState.TASK_FINISHED)
-        backend.statusUpdate(driver, status)
-      }
-    }.start
-
-    backend.stop()
-    // Any method of the backend involving sending messages to the driver endpoint should not
-    // be called after the backend is stopped.
-    verify(driverEndpoint, never()).askWithRetry(isA(classOf[RemoveExecutor]))(any[ClassTag[_]])
-  }
-
-  test("mesos supports spark.executor.uri") {
-    val url = "spark.spark.spark.com"
-    setBackend(Map(
-      "spark.executor.uri" -> url
-    ), false)
-
-    val (mem, cpu) = (backend.executorMemory(sc), 4)
-
-    val offer1 = createOffer("o1", "s1", mem, cpu)
-    backend.resourceOffers(driver, List(offer1).asJava)
-
-    val launchedTasks = verifyTaskLaunched(driver, "o1")
-    assert(launchedTasks.head.getCommand.getUrisList.asScala(0).getValue == url)
-  }
-
-  test("mesos supports setting fetcher cache") {
-    val url = "spark.spark.spark.com"
-    setBackend(Map(
-      "spark.mesos.fetcherCache.enable" -> "true",
-      "spark.executor.uri" -> url
-    ), false)
-    val offers = List(Resources(backend.executorMemory(sc), 1))
-    offerResources(offers)
-    val launchedTasks = verifyTaskLaunched(driver, "o1")
-    val uris = launchedTasks.head.getCommand.getUrisList
-    assert(uris.size() == 1)
-    assert(uris.asScala.head.getCache)
-  }
-
-  test("mesos supports disabling fetcher cache") {
-    val url = "spark.spark.spark.com"
-    setBackend(Map(
-      "spark.mesos.fetcherCache.enable" -> "false",
-      "spark.executor.uri" -> url
-    ), false)
-    val offers = List(Resources(backend.executorMemory(sc), 1))
-    offerResources(offers)
-    val launchedTasks = verifyTaskLaunched(driver, "o1")
-    val uris = launchedTasks.head.getCommand.getUrisList
-    assert(uris.size() == 1)
-    assert(!uris.asScala.head.getCache)
-  }
-
-  test("mesos supports spark.mesos.network.name") {
-    setBackend(Map(
-      "spark.mesos.network.name" -> "test-network-name"
-    ))
-
-    val (mem, cpu) = (backend.executorMemory(sc), 4)
-
-    val offer1 = createOffer("o1", "s1", mem, cpu)
-    backend.resourceOffers(driver, List(offer1).asJava)
-
-    val launchedTasks = verifyTaskLaunched(driver, "o1")
-    val networkInfos = launchedTasks.head.getContainer.getNetworkInfosList
-    assert(networkInfos.size == 1)
-    assert(networkInfos.get(0).getName == "test-network-name")
-  }
-
-  private case class Resources(mem: Int, cpus: Int, gpus: Int = 0)
-
-  private def verifyDeclinedOffer(driver: SchedulerDriver,
-      offerId: OfferID,
-      filter: Boolean = false): Unit = {
-    if (filter) {
-      verify(driver, times(1)).declineOffer(Matchers.eq(offerId), anyObject[Filters])
-    } else {
-      verify(driver, times(1)).declineOffer(Matchers.eq(offerId))
-    }
-  }
-
-  private def offerResources(offers: List[Resources], startId: Int = 1): Unit = {
-    val mesosOffers = offers.zipWithIndex.map {case (offer, i) =>
-      createOffer(s"o${i + startId}", s"s${i + startId}", offer.mem, offer.cpus, None, offer.gpus)}
-
-    backend.resourceOffers(driver, mesosOffers.asJava)
-  }
-
-  private def createTaskStatus(taskId: String, slaveId: String, state: TaskState): TaskStatus = {
-    TaskStatus.newBuilder()
-      .setTaskId(TaskID.newBuilder().setValue(taskId).build())
-      .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build())
-      .setState(state)
-      .build
-  }
-
-  private def createSchedulerBackend(
-      taskScheduler: TaskSchedulerImpl,
-      driver: SchedulerDriver,
-      shuffleClient: MesosExternalShuffleClient,
-      endpoint: RpcEndpointRef): MesosCoarseGrainedSchedulerBackend = {
-    val securityManager = mock[SecurityManager]
-
-    val backend = new MesosCoarseGrainedSchedulerBackend(
-        taskScheduler, sc, "master", securityManager) {
-      override protected def createSchedulerDriver(
-          masterUrl: String,
-          scheduler: Scheduler,
-          sparkUser: String,
-          appName: String,
-          conf: SparkConf,
-          webuiUrl: Option[String] = None,
-          checkpoint: Option[Boolean] = None,
-          failoverTimeout: Option[Double] = None,
-          frameworkId: Option[String] = None): SchedulerDriver = driver
-
-      override protected def getShuffleClient(): MesosExternalShuffleClient = shuffleClient
-
-      override protected def createDriverEndpointRef(
-          properties: ArrayBuffer[(String, String)]): RpcEndpointRef = endpoint
-
-      // override to avoid race condition with the driver thread on `mesosDriver`
-      override def startScheduler(newDriver: SchedulerDriver): Unit = {
-        mesosDriver = newDriver
-      }
-
-      override def stopExecutors(): Unit = {
-        stopCalled = true
-      }
-
-      markRegistered()
-    }
-    backend.start()
-    backend
-  }
-
-  private def setBackend(sparkConfVars: Map[String, String] = null,
-      setHome: Boolean = true) {
-    sparkConf = (new SparkConf)
-      .setMaster("local[*]")
-      .setAppName("test-mesos-dynamic-alloc")
-      .set("spark.mesos.driver.webui.url", "http://webui")
-
-    if (setHome) {
-      sparkConf.setSparkHome("/path")
-    }
-
-    if (sparkConfVars != null) {
-      sparkConf.setAll(sparkConfVars)
-    }
-
-    sc = new SparkContext(sparkConf)
-
-    driver = mock[SchedulerDriver]
-    when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING)
-    taskScheduler = mock[TaskSchedulerImpl]
-    when(taskScheduler.sc).thenReturn(sc)
-    externalShuffleClient = mock[MesosExternalShuffleClient]
-    driverEndpoint = mock[RpcEndpointRef]
-    when(driverEndpoint.ask(any())(any())).thenReturn(Promise().future)
-
-    backend = createSchedulerBackend(taskScheduler, driver, externalShuffleClient, driverEndpoint)
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala
----------------------------------------------------------------------
diff --git a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala
deleted file mode 100644
index 1d7a86f..0000000
--- a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala
+++ /dev/null
@@ -1,385 +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.mesos
-
-import java.nio.ByteBuffer
-import java.util.Arrays
-import java.util.Collection
-import java.util.Collections
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.mesos.{Protos, Scheduler, SchedulerDriver}
-import org.apache.mesos.Protos._
-import org.apache.mesos.Protos.Value.Scalar
-import org.mockito.{ArgumentCaptor, Matchers}
-import org.mockito.Matchers._
-import org.mockito.Mockito._
-import org.scalatest.mock.MockitoSugar
-
-import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite}
-import org.apache.spark.executor.MesosExecutorBackend
-import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded,
-  TaskDescription, TaskSchedulerImpl, WorkerOffer}
-import org.apache.spark.scheduler.cluster.ExecutorInfo
-
-class MesosFineGrainedSchedulerBackendSuite
-  extends SparkFunSuite with LocalSparkContext with MockitoSugar {
-
-  test("weburi is set in created scheduler driver") {
-    val conf = new SparkConf
-    conf.set("spark.mesos.driver.webui.url", "http://webui")
-    conf.set("spark.app.name", "name1")
-
-    val sc = mock[SparkContext]
-    when(sc.conf).thenReturn(conf)
-    when(sc.sparkUser).thenReturn("sparkUser1")
-    when(sc.appName).thenReturn("appName1")
-
-    val taskScheduler = mock[TaskSchedulerImpl]
-    val driver = mock[SchedulerDriver]
-    when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING)
-
-    val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") {
-      override protected def createSchedulerDriver(
-        masterUrl: String,
-        scheduler: Scheduler,
-        sparkUser: String,
-        appName: String,
-        conf: SparkConf,
-        webuiUrl: Option[String] = None,
-        checkpoint: Option[Boolean] = None,
-        failoverTimeout: Option[Double] = None,
-        frameworkId: Option[String] = None): SchedulerDriver = {
-        markRegistered()
-        assert(webuiUrl.isDefined)
-        assert(webuiUrl.get.equals("http://webui"))
-        driver
-      }
-    }
-
-    backend.start()
-  }
-
-  test("Use configured mesosExecutor.cores for ExecutorInfo") {
-    val mesosExecutorCores = 3
-    val conf = new SparkConf
-    conf.set("spark.mesos.mesosExecutor.cores", mesosExecutorCores.toString)
-
-    val listenerBus = mock[LiveListenerBus]
-    listenerBus.post(
-      SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty)))
-
-    val sc = mock[SparkContext]
-    when(sc.getSparkHome()).thenReturn(Option("/spark-home"))
-
-    when(sc.conf).thenReturn(conf)
-    when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String])
-    when(sc.executorMemory).thenReturn(100)
-    when(sc.listenerBus).thenReturn(listenerBus)
-    val taskScheduler = mock[TaskSchedulerImpl]
-    when(taskScheduler.CPUS_PER_TASK).thenReturn(2)
-
-    val mesosSchedulerBackend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master")
-
-    val resources = Arrays.asList(
-      mesosSchedulerBackend.createResource("cpus", 4),
-      mesosSchedulerBackend.createResource("mem", 1024))
-    // uri is null.
-    val (executorInfo, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id")
-    val executorResources = executorInfo.getResourcesList
-    val cpus = executorResources.asScala.find(_.getName.equals("cpus")).get.getScalar.getValue
-
-    assert(cpus === mesosExecutorCores)
-  }
-
-  test("check spark-class location correctly") {
-    val conf = new SparkConf
-    conf.set("spark.mesos.executor.home", "/mesos-home")
-
-    val listenerBus = mock[LiveListenerBus]
-    listenerBus.post(
-      SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty)))
-
-    val sc = mock[SparkContext]
-    when(sc.getSparkHome()).thenReturn(Option("/spark-home"))
-
-    when(sc.conf).thenReturn(conf)
-    when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String])
-    when(sc.executorMemory).thenReturn(100)
-    when(sc.listenerBus).thenReturn(listenerBus)
-    val taskScheduler = mock[TaskSchedulerImpl]
-    when(taskScheduler.CPUS_PER_TASK).thenReturn(2)
-
-    val mesosSchedulerBackend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master")
-
-    val resources = Arrays.asList(
-      mesosSchedulerBackend.createResource("cpus", 4),
-      mesosSchedulerBackend.createResource("mem", 1024))
-    // uri is null.
-    val (executorInfo, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id")
-    assert(executorInfo.getCommand.getValue ===
-      s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}")
-
-    // uri exists.
-    conf.set("spark.executor.uri", "hdfs:///test-app-1.0.0.tgz")
-    val (executorInfo1, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id")
-    assert(executorInfo1.getCommand.getValue ===
-      s"cd test-app-1*;  ./bin/spark-class ${classOf[MesosExecutorBackend].getName}")
-  }
-
-  test("spark docker properties correctly populate the DockerInfo message") {
-    val taskScheduler = mock[TaskSchedulerImpl]
-
-    val conf = new SparkConf()
-      .set("spark.mesos.executor.docker.image", "spark/mock")
-      .set("spark.mesos.executor.docker.forcePullImage", "true")
-      .set("spark.mesos.executor.docker.volumes", "/a,/b:/b,/c:/c:rw,/d:ro,/e:/e:ro")
-      .set("spark.mesos.executor.docker.portmaps", "80:8080,53:53:tcp")
-
-    val listenerBus = mock[LiveListenerBus]
-    listenerBus.post(
-      SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty)))
-
-    val sc = mock[SparkContext]
-    when(sc.executorMemory).thenReturn(100)
-    when(sc.getSparkHome()).thenReturn(Option("/spark-home"))
-    when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String])
-    when(sc.conf).thenReturn(conf)
-    when(sc.listenerBus).thenReturn(listenerBus)
-
-    val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master")
-
-    val (execInfo, _) = backend.createExecutorInfo(
-      Arrays.asList(backend.createResource("cpus", 4)), "mockExecutor")
-    assert(execInfo.getContainer.getDocker.getImage.equals("spark/mock"))
-    assert(execInfo.getContainer.getDocker.getForcePullImage.equals(true))
-    val portmaps = execInfo.getContainer.getDocker.getPortMappingsList
-    assert(portmaps.get(0).getHostPort.equals(80))
-    assert(portmaps.get(0).getContainerPort.equals(8080))
-    assert(portmaps.get(0).getProtocol.equals("tcp"))
-    assert(portmaps.get(1).getHostPort.equals(53))
-    assert(portmaps.get(1).getContainerPort.equals(53))
-    assert(portmaps.get(1).getProtocol.equals("tcp"))
-    val volumes = execInfo.getContainer.getVolumesList
-    assert(volumes.get(0).getContainerPath.equals("/a"))
-    assert(volumes.get(0).getMode.equals(Volume.Mode.RW))
-    assert(volumes.get(1).getContainerPath.equals("/b"))
-    assert(volumes.get(1).getHostPath.equals("/b"))
-    assert(volumes.get(1).getMode.equals(Volume.Mode.RW))
-    assert(volumes.get(2).getContainerPath.equals("/c"))
-    assert(volumes.get(2).getHostPath.equals("/c"))
-    assert(volumes.get(2).getMode.equals(Volume.Mode.RW))
-    assert(volumes.get(3).getContainerPath.equals("/d"))
-    assert(volumes.get(3).getMode.equals(Volume.Mode.RO))
-    assert(volumes.get(4).getContainerPath.equals("/e"))
-    assert(volumes.get(4).getHostPath.equals("/e"))
-    assert(volumes.get(4).getMode.equals(Volume.Mode.RO))
-  }
-
-  test("mesos resource offers result in launching tasks") {
-    def createOffer(id: Int, mem: Int, cpu: Int): Offer = {
-      val builder = Offer.newBuilder()
-      builder.addResourcesBuilder()
-        .setName("mem")
-        .setType(Value.Type.SCALAR)
-        .setScalar(Scalar.newBuilder().setValue(mem))
-      builder.addResourcesBuilder()
-        .setName("cpus")
-        .setType(Value.Type.SCALAR)
-        .setScalar(Scalar.newBuilder().setValue(cpu))
-      builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build())
-        .setFrameworkId(FrameworkID.newBuilder().setValue("f1"))
-        .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}"))
-        .setHostname(s"host${id.toString}").build()
-    }
-
-    val driver = mock[SchedulerDriver]
-    val taskScheduler = mock[TaskSchedulerImpl]
-
-    val listenerBus = mock[LiveListenerBus]
-    listenerBus.post(
-      SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty)))
-
-    val sc = mock[SparkContext]
-    when(sc.executorMemory).thenReturn(100)
-    when(sc.getSparkHome()).thenReturn(Option("/path"))
-    when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String])
-    when(sc.conf).thenReturn(new SparkConf)
-    when(sc.listenerBus).thenReturn(listenerBus)
-
-    val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master")
-
-    val minMem = backend.executorMemory(sc)
-    val minCpu = 4
-
-    val mesosOffers = new java.util.ArrayList[Offer]
-    mesosOffers.add(createOffer(1, minMem, minCpu))
-    mesosOffers.add(createOffer(2, minMem - 1, minCpu))
-    mesosOffers.add(createOffer(3, minMem, minCpu))
-
-    val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2)
-    expectedWorkerOffers += new WorkerOffer(
-      mesosOffers.get(0).getSlaveId.getValue,
-      mesosOffers.get(0).getHostname,
-      (minCpu - backend.mesosExecutorCores).toInt
-    )
-    expectedWorkerOffers += new WorkerOffer(
-      mesosOffers.get(2).getSlaveId.getValue,
-      mesosOffers.get(2).getHostname,
-      (minCpu - backend.mesosExecutorCores).toInt
-    )
-    val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0)))
-    when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc)))
-    when(taskScheduler.CPUS_PER_TASK).thenReturn(2)
-
-    val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]])
-    when(
-      driver.launchTasks(
-        Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)),
-        capture.capture(),
-        any(classOf[Filters])
-      )
-    ).thenReturn(Status.valueOf(1))
-    when(driver.declineOffer(mesosOffers.get(1).getId)).thenReturn(Status.valueOf(1))
-    when(driver.declineOffer(mesosOffers.get(2).getId)).thenReturn(Status.valueOf(1))
-
-    backend.resourceOffers(driver, mesosOffers)
-
-    verify(driver, times(1)).launchTasks(
-      Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)),
-      capture.capture(),
-      any(classOf[Filters])
-    )
-    verify(driver, times(1)).declineOffer(mesosOffers.get(1).getId)
-    verify(driver, times(1)).declineOffer(mesosOffers.get(2).getId)
-    assert(capture.getValue.size() === 1)
-    val taskInfo = capture.getValue.iterator().next()
-    assert(taskInfo.getName.equals("n1"))
-    val cpus = taskInfo.getResourcesList.get(0)
-    assert(cpus.getName.equals("cpus"))
-    assert(cpus.getScalar.getValue.equals(2.0))
-    assert(taskInfo.getSlaveId.getValue.equals("s1"))
-
-    // Unwanted resources offered on an existing node. Make sure they are declined
-    val mesosOffers2 = new java.util.ArrayList[Offer]
-    mesosOffers2.add(createOffer(1, minMem, minCpu))
-    reset(taskScheduler)
-    reset(driver)
-    when(taskScheduler.resourceOffers(any(classOf[IndexedSeq[WorkerOffer]]))).thenReturn(Seq(Seq()))
-    when(taskScheduler.CPUS_PER_TASK).thenReturn(2)
-    when(driver.declineOffer(mesosOffers2.get(0).getId)).thenReturn(Status.valueOf(1))
-
-    backend.resourceOffers(driver, mesosOffers2)
-    verify(driver, times(1)).declineOffer(mesosOffers2.get(0).getId)
-  }
-
-  test("can handle multiple roles") {
-    val driver = mock[SchedulerDriver]
-    val taskScheduler = mock[TaskSchedulerImpl]
-
-    val listenerBus = mock[LiveListenerBus]
-    listenerBus.post(
-      SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty)))
-
-    val sc = mock[SparkContext]
-    when(sc.executorMemory).thenReturn(100)
-    when(sc.getSparkHome()).thenReturn(Option("/path"))
-    when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String])
-    when(sc.conf).thenReturn(new SparkConf)
-    when(sc.listenerBus).thenReturn(listenerBus)
-
-    val id = 1
-    val builder = Offer.newBuilder()
-    builder.addResourcesBuilder()
-      .setName("mem")
-      .setType(Value.Type.SCALAR)
-      .setRole("prod")
-      .setScalar(Scalar.newBuilder().setValue(500))
-    builder.addResourcesBuilder()
-      .setName("cpus")
-      .setRole("prod")
-      .setType(Value.Type.SCALAR)
-      .setScalar(Scalar.newBuilder().setValue(1))
-    builder.addResourcesBuilder()
-      .setName("mem")
-      .setRole("dev")
-      .setType(Value.Type.SCALAR)
-      .setScalar(Scalar.newBuilder().setValue(600))
-    builder.addResourcesBuilder()
-      .setName("cpus")
-      .setRole("dev")
-      .setType(Value.Type.SCALAR)
-      .setScalar(Scalar.newBuilder().setValue(2))
-    val offer = builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build())
-      .setFrameworkId(FrameworkID.newBuilder().setValue("f1"))
-      .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}"))
-      .setHostname(s"host${id.toString}").build()
-
-    val mesosOffers = new java.util.ArrayList[Offer]
-    mesosOffers.add(offer)
-
-    val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master")
-
-    val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](1)
-    expectedWorkerOffers += new WorkerOffer(
-      mesosOffers.get(0).getSlaveId.getValue,
-      mesosOffers.get(0).getHostname,
-      2 // Deducting 1 for executor
-    )
-
-    val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0)))
-    when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc)))
-    when(taskScheduler.CPUS_PER_TASK).thenReturn(1)
-
-    val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]])
-    when(
-      driver.launchTasks(
-        Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)),
-        capture.capture(),
-        any(classOf[Filters])
-      )
-    ).thenReturn(Status.valueOf(1))
-
-    backend.resourceOffers(driver, mesosOffers)
-
-    verify(driver, times(1)).launchTasks(
-      Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)),
-      capture.capture(),
-      any(classOf[Filters])
-    )
-
-    assert(capture.getValue.size() === 1)
-    val taskInfo = capture.getValue.iterator().next()
-    assert(taskInfo.getName.equals("n1"))
-    assert(taskInfo.getResourcesCount === 1)
-    val cpusDev = taskInfo.getResourcesList.get(0)
-    assert(cpusDev.getName.equals("cpus"))
-    assert(cpusDev.getScalar.getValue.equals(1.0))
-    assert(cpusDev.getRole.equals("dev"))
-    val executorResources = taskInfo.getExecutor.getResourcesList.asScala
-    assert(executorResources.exists { r =>
-      r.getName.equals("mem") && r.getScalar.getValue.equals(484.0) && r.getRole.equals("prod")
-    })
-    assert(executorResources.exists { r =>
-      r.getName.equals("cpus") && r.getScalar.getValue.equals(1.0) && r.getRole.equals("prod")
-    })
-  }
-}


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


[13/18] spark git commit: [SPARK-18662] Move resource managers to separate directory

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
new file mode 100644
index 0000000..1d742fe
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
@@ -0,0 +1,524 @@
+/*
+ * 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.mesos
+
+import java.util.{List => JList}
+import java.util.concurrent.CountDownLatch
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+import scala.util.control.NonFatal
+
+import com.google.common.base.Splitter
+import org.apache.mesos.{MesosSchedulerDriver, Protos, Scheduler, SchedulerDriver}
+import org.apache.mesos.Protos.{TaskState => MesosTaskState, _}
+import org.apache.mesos.Protos.FrameworkInfo.Capability
+import org.apache.mesos.protobuf.{ByteString, GeneratedMessage}
+
+import org.apache.spark.{SparkConf, SparkContext, SparkException}
+import org.apache.spark.TaskState
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.util.Utils
+
+
+
+/**
+ * Shared trait for implementing a Mesos Scheduler. This holds common state and helper
+ * methods and Mesos scheduler will use.
+ */
+trait MesosSchedulerUtils extends Logging {
+  // Lock used to wait for scheduler to be registered
+  private final val registerLatch = new CountDownLatch(1)
+
+  // Driver for talking to Mesos
+  protected var mesosDriver: SchedulerDriver = null
+
+  /**
+   * Creates a new MesosSchedulerDriver that communicates to the Mesos master.
+   *
+   * @param masterUrl The url to connect to Mesos master
+   * @param scheduler the scheduler class to receive scheduler callbacks
+   * @param sparkUser User to impersonate with when running tasks
+   * @param appName The framework name to display on the Mesos UI
+   * @param conf Spark configuration
+   * @param webuiUrl The WebUI url to link from Mesos UI
+   * @param checkpoint Option to checkpoint tasks for failover
+   * @param failoverTimeout Duration Mesos master expect scheduler to reconnect on disconnect
+   * @param frameworkId The id of the new framework
+   */
+  protected def createSchedulerDriver(
+      masterUrl: String,
+      scheduler: Scheduler,
+      sparkUser: String,
+      appName: String,
+      conf: SparkConf,
+      webuiUrl: Option[String] = None,
+      checkpoint: Option[Boolean] = None,
+      failoverTimeout: Option[Double] = None,
+      frameworkId: Option[String] = None): SchedulerDriver = {
+    val fwInfoBuilder = FrameworkInfo.newBuilder().setUser(sparkUser).setName(appName)
+    val credBuilder = Credential.newBuilder()
+    webuiUrl.foreach { url => fwInfoBuilder.setWebuiUrl(url) }
+    checkpoint.foreach { checkpoint => fwInfoBuilder.setCheckpoint(checkpoint) }
+    failoverTimeout.foreach { timeout => fwInfoBuilder.setFailoverTimeout(timeout) }
+    frameworkId.foreach { id =>
+      fwInfoBuilder.setId(FrameworkID.newBuilder().setValue(id).build())
+    }
+    fwInfoBuilder.setHostname(Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(
+      conf.get(DRIVER_HOST_ADDRESS)))
+    conf.getOption("spark.mesos.principal").foreach { principal =>
+      fwInfoBuilder.setPrincipal(principal)
+      credBuilder.setPrincipal(principal)
+    }
+    conf.getOption("spark.mesos.secret").foreach { secret =>
+      credBuilder.setSecret(secret)
+    }
+    if (credBuilder.hasSecret && !fwInfoBuilder.hasPrincipal) {
+      throw new SparkException(
+        "spark.mesos.principal must be configured when spark.mesos.secret is set")
+    }
+    conf.getOption("spark.mesos.role").foreach { role =>
+      fwInfoBuilder.setRole(role)
+    }
+    val maxGpus = conf.getInt("spark.mesos.gpus.max", 0)
+    if (maxGpus > 0) {
+      fwInfoBuilder.addCapabilities(Capability.newBuilder().setType(Capability.Type.GPU_RESOURCES))
+    }
+    if (credBuilder.hasPrincipal) {
+      new MesosSchedulerDriver(
+        scheduler, fwInfoBuilder.build(), masterUrl, credBuilder.build())
+    } else {
+      new MesosSchedulerDriver(scheduler, fwInfoBuilder.build(), masterUrl)
+    }
+  }
+
+  /**
+   * Starts the MesosSchedulerDriver and stores the current running driver to this new instance.
+   * This driver is expected to not be running.
+   * This method returns only after the scheduler has registered with Mesos.
+   */
+  def startScheduler(newDriver: SchedulerDriver): Unit = {
+    synchronized {
+      if (mesosDriver != null) {
+        registerLatch.await()
+        return
+      }
+      @volatile
+      var error: Option[Exception] = None
+
+      // We create a new thread that will block inside `mesosDriver.run`
+      // until the scheduler exists
+      new Thread(Utils.getFormattedClassName(this) + "-mesos-driver") {
+        setDaemon(true)
+        override def run() {
+          try {
+            mesosDriver = newDriver
+            val ret = mesosDriver.run()
+            logInfo("driver.run() returned with code " + ret)
+            if (ret != null && ret.equals(Status.DRIVER_ABORTED)) {
+              error = Some(new SparkException("Error starting driver, DRIVER_ABORTED"))
+              markErr()
+            }
+          } catch {
+            case e: Exception =>
+              logError("driver.run() failed", e)
+              error = Some(e)
+              markErr()
+          }
+        }
+      }.start()
+
+      registerLatch.await()
+
+      // propagate any error to the calling thread. This ensures that SparkContext creation fails
+      // without leaving a broken context that won't be able to schedule any tasks
+      error.foreach(throw _)
+    }
+  }
+
+  def getResource(res: JList[Resource], name: String): Double = {
+    // A resource can have multiple values in the offer since it can either be from
+    // a specific role or wildcard.
+    res.asScala.filter(_.getName == name).map(_.getScalar.getValue).sum
+  }
+
+  /**
+   * Transforms a range resource to a list of ranges
+   *
+   * @param res the mesos resource list
+   * @param name the name of the resource
+   * @return the list of ranges returned
+   */
+  protected def getRangeResource(res: JList[Resource], name: String): List[(Long, Long)] = {
+    // A resource can have multiple values in the offer since it can either be from
+    // a specific role or wildcard.
+    res.asScala.filter(_.getName == name).flatMap(_.getRanges.getRangeList.asScala
+      .map(r => (r.getBegin, r.getEnd)).toList).toList
+  }
+
+  /**
+   * Signal that the scheduler has registered with Mesos.
+   */
+  protected def markRegistered(): Unit = {
+    registerLatch.countDown()
+  }
+
+  protected def markErr(): Unit = {
+    registerLatch.countDown()
+  }
+
+  def createResource(name: String, amount: Double, role: Option[String] = None): Resource = {
+    val builder = Resource.newBuilder()
+      .setName(name)
+      .setType(Value.Type.SCALAR)
+      .setScalar(Value.Scalar.newBuilder().setValue(amount).build())
+
+    role.foreach { r => builder.setRole(r) }
+
+    builder.build()
+  }
+
+  /**
+   * Partition the existing set of resources into two groups, those remaining to be
+   * scheduled and those requested to be used for a new task.
+   *
+   * @param resources The full list of available resources
+   * @param resourceName The name of the resource to take from the available resources
+   * @param amountToUse The amount of resources to take from the available resources
+   * @return The remaining resources list and the used resources list.
+   */
+  def partitionResources(
+      resources: JList[Resource],
+      resourceName: String,
+      amountToUse: Double): (List[Resource], List[Resource]) = {
+    var remain = amountToUse
+    var requestedResources = new ArrayBuffer[Resource]
+    val remainingResources = resources.asScala.map {
+      case r =>
+        if (remain > 0 &&
+          r.getType == Value.Type.SCALAR &&
+          r.getScalar.getValue > 0.0 &&
+          r.getName == resourceName) {
+          val usage = Math.min(remain, r.getScalar.getValue)
+          requestedResources += createResource(resourceName, usage, Some(r.getRole))
+          remain -= usage
+          createResource(resourceName, r.getScalar.getValue - usage, Some(r.getRole))
+        } else {
+          r
+        }
+    }
+
+    // Filter any resource that has depleted.
+    val filteredResources =
+      remainingResources.filter(r => r.getType != Value.Type.SCALAR || r.getScalar.getValue > 0.0)
+
+    (filteredResources.toList, requestedResources.toList)
+  }
+
+  /** Helper method to get the key,value-set pair for a Mesos Attribute protobuf */
+  protected def getAttribute(attr: Attribute): (String, Set[String]) = {
+    (attr.getName, attr.getText.getValue.split(',').toSet)
+  }
+
+
+  /** Build a Mesos resource protobuf object */
+  protected def createResource(resourceName: String, quantity: Double): Protos.Resource = {
+    Resource.newBuilder()
+      .setName(resourceName)
+      .setType(Value.Type.SCALAR)
+      .setScalar(Value.Scalar.newBuilder().setValue(quantity).build())
+      .build()
+  }
+
+  /**
+   * Converts the attributes from the resource offer into a Map of name -> Attribute Value
+   * The attribute values are the mesos attribute types and they are
+   *
+   * @param offerAttributes the attributes offered
+   * @return
+   */
+  protected def toAttributeMap(offerAttributes: JList[Attribute]): Map[String, GeneratedMessage] = {
+    offerAttributes.asScala.map { attr =>
+      val attrValue = attr.getType match {
+        case Value.Type.SCALAR => attr.getScalar
+        case Value.Type.RANGES => attr.getRanges
+        case Value.Type.SET => attr.getSet
+        case Value.Type.TEXT => attr.getText
+      }
+      (attr.getName, attrValue)
+    }.toMap
+  }
+
+
+  /**
+   * Match the requirements (if any) to the offer attributes.
+   * if attribute requirements are not specified - return true
+   * else if attribute is defined and no values are given, simple attribute presence is performed
+   * else if attribute name and value is specified, subset match is performed on slave attributes
+   */
+  def matchesAttributeRequirements(
+      slaveOfferConstraints: Map[String, Set[String]],
+      offerAttributes: Map[String, GeneratedMessage]): Boolean = {
+    slaveOfferConstraints.forall {
+      // offer has the required attribute and subsumes the required values for that attribute
+      case (name, requiredValues) =>
+        offerAttributes.get(name) match {
+          case None => false
+          case Some(_) if requiredValues.isEmpty => true // empty value matches presence
+          case Some(scalarValue: Value.Scalar) =>
+            // check if provided values is less than equal to the offered values
+            requiredValues.map(_.toDouble).exists(_ <= scalarValue.getValue)
+          case Some(rangeValue: Value.Range) =>
+            val offerRange = rangeValue.getBegin to rangeValue.getEnd
+            // Check if there is some required value that is between the ranges specified
+            // Note: We only support the ability to specify discrete values, in the future
+            // we may expand it to subsume ranges specified with a XX..YY value or something
+            // similar to that.
+            requiredValues.map(_.toLong).exists(offerRange.contains(_))
+          case Some(offeredValue: Value.Set) =>
+            // check if the specified required values is a subset of offered set
+            requiredValues.subsetOf(offeredValue.getItemList.asScala.toSet)
+          case Some(textValue: Value.Text) =>
+            // check if the specified value is equal, if multiple values are specified
+            // we succeed if any of them match.
+            requiredValues.contains(textValue.getValue)
+        }
+    }
+  }
+
+  /**
+   * Parses the attributes constraints provided to spark and build a matching data struct:
+   *  Map[<attribute-name>, Set[values-to-match]]
+   *  The constraints are specified as ';' separated key-value pairs where keys and values
+   *  are separated by ':'. The ':' implies equality (for singular values) and "is one of" for
+   *  multiple values (comma separated). For example:
+   *  {{{
+   *  parseConstraintString("os:centos7;zone:us-east-1a,us-east-1b")
+   *  // would result in
+   *  <code>
+   *  Map(
+   *    "os" -> Set("centos7"),
+   *    "zone":   -> Set("us-east-1a", "us-east-1b")
+   *  )
+   *  }}}
+   *
+   *  Mesos documentation: http://mesos.apache.org/documentation/attributes-resources/
+   *                       https://github.com/apache/mesos/blob/master/src/common/values.cpp
+   *                       https://github.com/apache/mesos/blob/master/src/common/attributes.cpp
+   *
+   * @param constraintsVal constaints string consisting of ';' separated key-value pairs (separated
+   *                       by ':')
+   * @return  Map of constraints to match resources offers.
+   */
+  def parseConstraintString(constraintsVal: String): Map[String, Set[String]] = {
+    /*
+      Based on mesos docs:
+      attributes : attribute ( ";" attribute )*
+      attribute : labelString ":" ( labelString | "," )+
+      labelString : [a-zA-Z0-9_/.-]
+    */
+    val splitter = Splitter.on(';').trimResults().withKeyValueSeparator(':')
+    // kv splitter
+    if (constraintsVal.isEmpty) {
+      Map()
+    } else {
+      try {
+        splitter.split(constraintsVal).asScala.toMap.mapValues(v =>
+          if (v == null || v.isEmpty) {
+            Set[String]()
+          } else {
+            v.split(',').toSet
+          }
+        )
+      } catch {
+        case NonFatal(e) =>
+          throw new IllegalArgumentException(s"Bad constraint string: $constraintsVal", e)
+      }
+    }
+  }
+
+  // These defaults copied from YARN
+  private val MEMORY_OVERHEAD_FRACTION = 0.10
+  private val MEMORY_OVERHEAD_MINIMUM = 384
+
+  /**
+   * Return the amount of memory to allocate to each executor, taking into account
+   * container overheads.
+   *
+   * @param sc SparkContext to use to get `spark.mesos.executor.memoryOverhead` value
+   * @return memory requirement as (0.1 * <memoryOverhead>) or MEMORY_OVERHEAD_MINIMUM
+   *         (whichever is larger)
+   */
+  def executorMemory(sc: SparkContext): Int = {
+    sc.conf.getInt("spark.mesos.executor.memoryOverhead",
+      math.max(MEMORY_OVERHEAD_FRACTION * sc.executorMemory, MEMORY_OVERHEAD_MINIMUM).toInt) +
+      sc.executorMemory
+  }
+
+  def setupUris(uris: String,
+                builder: CommandInfo.Builder,
+                useFetcherCache: Boolean = false): Unit = {
+    uris.split(",").foreach { uri =>
+      builder.addUris(CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetcherCache))
+    }
+  }
+
+  protected def getRejectOfferDurationForUnmetConstraints(sc: SparkContext): Long = {
+    sc.conf.getTimeAsSeconds("spark.mesos.rejectOfferDurationForUnmetConstraints", "120s")
+  }
+
+  protected def getRejectOfferDurationForReachedMaxCores(sc: SparkContext): Long = {
+    sc.conf.getTimeAsSeconds("spark.mesos.rejectOfferDurationForReachedMaxCores", "120s")
+  }
+
+  /**
+   * Checks executor ports if they are within some range of the offered list of ports ranges,
+   *
+   * @param conf the Spark Config
+   * @param ports the list of ports to check
+   * @return true if ports are within range false otherwise
+   */
+  protected def checkPorts(conf: SparkConf, ports: List[(Long, Long)]): Boolean = {
+
+    def checkIfInRange(port: Long, ps: List[(Long, Long)]): Boolean = {
+      ps.exists{case (rangeStart, rangeEnd) => rangeStart <= port & rangeEnd >= port }
+    }
+
+    val portsToCheck = nonZeroPortValuesFromConfig(conf)
+    val withinRange = portsToCheck.forall(p => checkIfInRange(p, ports))
+    // make sure we have enough ports to allocate per offer
+    val enoughPorts =
+    ports.map{case (rangeStart, rangeEnd) => rangeEnd - rangeStart + 1}.sum >= portsToCheck.size
+    enoughPorts && withinRange
+  }
+
+  /**
+   * Partitions port resources.
+   *
+   * @param requestedPorts non-zero ports to assign
+   * @param offeredResources the resources offered
+   * @return resources left, port resources to be used.
+   */
+  def partitionPortResources(requestedPorts: List[Long], offeredResources: List[Resource])
+    : (List[Resource], List[Resource]) = {
+    if (requestedPorts.isEmpty) {
+      (offeredResources, List[Resource]())
+    } else {
+      // partition port offers
+      val (resourcesWithoutPorts, portResources) = filterPortResources(offeredResources)
+
+      val portsAndRoles = requestedPorts.
+        map(x => (x, findPortAndGetAssignedRangeRole(x, portResources)))
+
+      val assignedPortResources = createResourcesFromPorts(portsAndRoles)
+
+      // ignore non-assigned port resources, they will be declined implicitly by mesos
+      // no need for splitting port resources.
+      (resourcesWithoutPorts, assignedPortResources)
+    }
+  }
+
+  val managedPortNames = List("spark.executor.port", BLOCK_MANAGER_PORT.key)
+
+  /**
+   * The values of the non-zero ports to be used by the executor process.
+   * @param conf the spark config to use
+   * @return the ono-zero values of the ports
+   */
+  def nonZeroPortValuesFromConfig(conf: SparkConf): List[Long] = {
+    managedPortNames.map(conf.getLong(_, 0)).filter( _ != 0)
+  }
+
+  /** Creates a mesos resource for a specific port number. */
+  private def createResourcesFromPorts(portsAndRoles: List[(Long, String)]) : List[Resource] = {
+    portsAndRoles.flatMap{ case (port, role) =>
+      createMesosPortResource(List((port, port)), Some(role))}
+  }
+
+  /** Helper to create mesos resources for specific port ranges. */
+  private def createMesosPortResource(
+      ranges: List[(Long, Long)],
+      role: Option[String] = None): List[Resource] = {
+    ranges.map { case (rangeStart, rangeEnd) =>
+      val rangeValue = Value.Range.newBuilder()
+        .setBegin(rangeStart)
+        .setEnd(rangeEnd)
+      val builder = Resource.newBuilder()
+        .setName("ports")
+        .setType(Value.Type.RANGES)
+        .setRanges(Value.Ranges.newBuilder().addRange(rangeValue))
+      role.foreach(r => builder.setRole(r))
+      builder.build()
+    }
+  }
+
+ /**
+  * Helper to assign a port to an offered range and get the latter's role
+  * info to use it later on.
+  */
+  private def findPortAndGetAssignedRangeRole(port: Long, portResources: List[Resource])
+    : String = {
+
+    val ranges = portResources.
+      map(resource =>
+        (resource.getRole, resource.getRanges.getRangeList.asScala
+          .map(r => (r.getBegin, r.getEnd)).toList))
+
+    val rangePortRole = ranges
+      .find { case (role, rangeList) => rangeList
+        .exists{ case (rangeStart, rangeEnd) => rangeStart <= port & rangeEnd >= port}}
+    // this is safe since we have previously checked about the ranges (see checkPorts method)
+    rangePortRole.map{ case (role, rangeList) => role}.get
+  }
+
+  /** Retrieves the port resources from a list of mesos offered resources */
+  private def filterPortResources(resources: List[Resource]): (List[Resource], List[Resource]) = {
+    resources.partition { r => !(r.getType == Value.Type.RANGES && r.getName == "ports") }
+  }
+
+  /**
+   * spark.mesos.driver.frameworkId is set by the cluster dispatcher to correlate driver
+   * submissions with frameworkIDs.  However, this causes issues when a driver process launches
+   * more than one framework (more than one SparkContext(, because they all try to register with
+   * the same frameworkID.  To enforce that only the first driver registers with the configured
+   * framework ID, the driver calls this method after the first registration.
+   */
+  def unsetFrameworkID(sc: SparkContext) {
+    sc.conf.remove("spark.mesos.driver.frameworkId")
+    System.clearProperty("spark.mesos.driver.frameworkId")
+  }
+
+  def mesosToTaskState(state: MesosTaskState): TaskState.TaskState = state match {
+    case MesosTaskState.TASK_STAGING | MesosTaskState.TASK_STARTING => TaskState.LAUNCHING
+    case MesosTaskState.TASK_RUNNING | MesosTaskState.TASK_KILLING => TaskState.RUNNING
+    case MesosTaskState.TASK_FINISHED => TaskState.FINISHED
+    case MesosTaskState.TASK_FAILED => TaskState.FAILED
+    case MesosTaskState.TASK_KILLED => TaskState.KILLED
+    case MesosTaskState.TASK_LOST | MesosTaskState.TASK_ERROR => TaskState.LOST
+  }
+
+  def taskStateToMesos(state: TaskState.TaskState): MesosTaskState = state match {
+    case TaskState.LAUNCHING => MesosTaskState.TASK_STARTING
+    case TaskState.RUNNING => MesosTaskState.TASK_RUNNING
+    case TaskState.FINISHED => MesosTaskState.TASK_FINISHED
+    case TaskState.FAILED => MesosTaskState.TASK_FAILED
+    case TaskState.KILLED => MesosTaskState.TASK_KILLED
+    case TaskState.LOST => MesosTaskState.TASK_LOST
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala
new file mode 100644
index 0000000..8370b61
--- /dev/null
+++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala
@@ -0,0 +1,51 @@
+/*
+ * 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.mesos
+
+import java.nio.ByteBuffer
+
+import org.apache.mesos.protobuf.ByteString
+
+import org.apache.spark.internal.Logging
+
+/**
+ * Wrapper for serializing the data sent when launching Mesos tasks.
+ */
+private[spark] case class MesosTaskLaunchData(
+  serializedTask: ByteBuffer,
+  attemptNumber: Int) extends Logging {
+
+  def toByteString: ByteString = {
+    val dataBuffer = ByteBuffer.allocate(4 + serializedTask.limit)
+    dataBuffer.putInt(attemptNumber)
+    dataBuffer.put(serializedTask)
+    dataBuffer.rewind
+    logDebug(s"ByteBuffer size: [${dataBuffer.remaining}]")
+    ByteString.copyFrom(dataBuffer)
+  }
+}
+
+private[spark] object MesosTaskLaunchData extends Logging {
+  def fromByteString(byteString: ByteString): MesosTaskLaunchData = {
+    val byteBuffer = byteString.asReadOnlyByteBuffer()
+    logDebug(s"ByteBuffer size: [${byteBuffer.remaining}]")
+    val attemptNumber = byteBuffer.getInt // updates the position by 4 bytes
+    val serializedTask = byteBuffer.slice() // subsequence starting at the current position
+    MesosTaskLaunchData(serializedTask, attemptNumber)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala
new file mode 100644
index 0000000..33e7d69
--- /dev/null
+++ b/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.mesos
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.TestPrematureExit
+
+class MesosClusterDispatcherArgumentsSuite extends SparkFunSuite
+  with TestPrematureExit {
+
+  test("test if spark config args are passed sucessfully") {
+    val args = Array[String]("--master", "mesos://localhost:5050", "--conf", "key1=value1",
+      "--conf", "spark.mesos.key2=value2", "--verbose")
+    val conf = new SparkConf()
+    new MesosClusterDispatcherArguments(args, conf)
+
+    assert(conf.getOption("key1").isEmpty)
+    assert(conf.get("spark.mesos.key2") == "value2")
+  }
+
+  test("test non conf settings") {
+    val masterUrl = "mesos://localhost:5050"
+    val port = "1212"
+    val zookeeperUrl = "zk://localhost:2181"
+    val host = "localhost"
+    val webUiPort = "2323"
+    val name = "myFramework"
+
+    val args1 = Array("--master", masterUrl, "--verbose", "--name", name)
+    val args2 = Array("-p", port, "-h", host, "-z", zookeeperUrl)
+    val args3 = Array("--webui-port", webUiPort)
+
+    val args = args1 ++ args2 ++ args3
+    val conf = new SparkConf()
+    val mesosDispClusterArgs = new MesosClusterDispatcherArguments(args, conf)
+
+    assert(mesosDispClusterArgs.verbose)
+    assert(mesosDispClusterArgs.confProperties.isEmpty)
+    assert(mesosDispClusterArgs.host == host)
+    assert(Option(mesosDispClusterArgs.masterUrl).isDefined)
+    assert(mesosDispClusterArgs.masterUrl == masterUrl.stripPrefix("mesos://"))
+    assert(Option(mesosDispClusterArgs.zookeeperUrl).isDefined)
+    assert(mesosDispClusterArgs.zookeeperUrl == Some(zookeeperUrl))
+    assert(mesosDispClusterArgs.name == name)
+    assert(mesosDispClusterArgs.webUiPort == webUiPort.toInt)
+    assert(mesosDispClusterArgs.port == port.toInt)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherSuite.scala
new file mode 100644
index 0000000..7484e3b
--- /dev/null
+++ b/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherSuite.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.mesos
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.deploy.TestPrematureExit
+
+class MesosClusterDispatcherSuite extends SparkFunSuite
+  with TestPrematureExit{
+
+  test("prints usage on empty input") {
+    testPrematureExit(Array[String](),
+      "Usage: MesosClusterDispatcher", MesosClusterDispatcher)
+  }
+
+  test("prints usage with only --help") {
+    testPrematureExit(Array("--help"),
+      "Usage: MesosClusterDispatcher", MesosClusterDispatcher)
+  }
+
+  test("prints error with unrecognized options") {
+    testPrematureExit(Array("--blarg"), "Unrecognized option: '--blarg'", MesosClusterDispatcher)
+    testPrematureExit(Array("-bleg"), "Unrecognized option: '-bleg'", MesosClusterDispatcher)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala
new file mode 100644
index 0000000..a558554
--- /dev/null
+++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala
@@ -0,0 +1,56 @@
+/*
+ * 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.mesos
+
+import org.apache.spark._
+import org.apache.spark.internal.config._
+
+class MesosClusterManagerSuite extends SparkFunSuite with LocalSparkContext {
+    def testURL(masterURL: String, expectedClass: Class[_], coarse: Boolean) {
+      val conf = new SparkConf().set("spark.mesos.coarse", coarse.toString)
+      sc = new SparkContext("local", "test", conf)
+      val clusterManager = new MesosClusterManager()
+
+      assert(clusterManager.canCreate(masterURL))
+      val taskScheduler = clusterManager.createTaskScheduler(sc, masterURL)
+      val sched = clusterManager.createSchedulerBackend(sc, masterURL, taskScheduler)
+      assert(sched.getClass === expectedClass)
+    }
+
+    test("mesos fine-grained") {
+      testURL("mesos://localhost:1234", classOf[MesosFineGrainedSchedulerBackend], coarse = false)
+    }
+
+    test("mesos coarse-grained") {
+      testURL("mesos://localhost:1234", classOf[MesosCoarseGrainedSchedulerBackend], coarse = true)
+    }
+
+    test("mesos with zookeeper") {
+      testURL("mesos://zk://localhost:1234,localhost:2345",
+          classOf[MesosFineGrainedSchedulerBackend],
+          coarse = false)
+    }
+
+    test("mesos with i/o encryption throws error") {
+      val se = intercept[SparkException] {
+        val conf = new SparkConf().setAppName("test").set(IO_ENCRYPTION_ENABLED, true)
+        sc = new SparkContext("mesos", "test", conf)
+      }
+      assert(se.getCause().isInstanceOf[IllegalArgumentException])
+    }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala
new file mode 100644
index 0000000..74e5ce2
--- /dev/null
+++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala
@@ -0,0 +1,239 @@
+/*
+ * 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.mesos
+
+import java.util.{Collection, Collections, Date}
+
+import scala.collection.JavaConverters._
+
+import org.apache.mesos.Protos._
+import org.apache.mesos.Protos.Value.{Scalar, Type}
+import org.apache.mesos.SchedulerDriver
+import org.mockito.{ArgumentCaptor, Matchers}
+import org.mockito.Mockito._
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.Command
+import org.apache.spark.deploy.mesos.MesosDriverDescription
+
+class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar {
+
+  private val command = new Command("mainClass", Seq("arg"), Map(), Seq(), Seq(), Seq())
+  private var driver: SchedulerDriver = _
+  private var scheduler: MesosClusterScheduler = _
+
+  private def setScheduler(sparkConfVars: Map[String, String] = null): Unit = {
+    val conf = new SparkConf()
+    conf.setMaster("mesos://localhost:5050")
+    conf.setAppName("spark mesos")
+
+    if (sparkConfVars != null) {
+      conf.setAll(sparkConfVars)
+    }
+
+    driver = mock[SchedulerDriver]
+    scheduler = new MesosClusterScheduler(
+      new BlackHoleMesosClusterPersistenceEngineFactory, conf) {
+      override def start(): Unit = { ready = true }
+    }
+    scheduler.start()
+  }
+
+  test("can queue drivers") {
+    setScheduler()
+
+    val response = scheduler.submitDriver(
+      new MesosDriverDescription("d1", "jar", 1000, 1, true,
+        command, Map[String, String](), "s1", new Date()))
+    assert(response.success)
+    val response2 =
+      scheduler.submitDriver(new MesosDriverDescription(
+        "d1", "jar", 1000, 1, true, command, Map[String, String](), "s2", new Date()))
+    assert(response2.success)
+    val state = scheduler.getSchedulerState()
+    val queuedDrivers = state.queuedDrivers.toList
+    assert(queuedDrivers(0).submissionId == response.submissionId)
+    assert(queuedDrivers(1).submissionId == response2.submissionId)
+  }
+
+  test("can kill queued drivers") {
+    setScheduler()
+
+    val response = scheduler.submitDriver(
+        new MesosDriverDescription("d1", "jar", 1000, 1, true,
+          command, Map[String, String](), "s1", new Date()))
+    assert(response.success)
+    val killResponse = scheduler.killDriver(response.submissionId)
+    assert(killResponse.success)
+    val state = scheduler.getSchedulerState()
+    assert(state.queuedDrivers.isEmpty)
+  }
+
+  test("can handle multiple roles") {
+    setScheduler()
+
+    val driver = mock[SchedulerDriver]
+    val response = scheduler.submitDriver(
+      new MesosDriverDescription("d1", "jar", 1200, 1.5, true,
+        command,
+        Map(("spark.mesos.executor.home", "test"), ("spark.app.name", "test")),
+        "s1",
+        new Date()))
+    assert(response.success)
+    val offer = Offer.newBuilder()
+      .addResources(
+        Resource.newBuilder().setRole("*")
+          .setScalar(Scalar.newBuilder().setValue(1).build()).setName("cpus").setType(Type.SCALAR))
+      .addResources(
+        Resource.newBuilder().setRole("*")
+          .setScalar(Scalar.newBuilder().setValue(1000).build())
+          .setName("mem")
+          .setType(Type.SCALAR))
+      .addResources(
+        Resource.newBuilder().setRole("role2")
+          .setScalar(Scalar.newBuilder().setValue(1).build()).setName("cpus").setType(Type.SCALAR))
+      .addResources(
+        Resource.newBuilder().setRole("role2")
+          .setScalar(Scalar.newBuilder().setValue(500).build()).setName("mem").setType(Type.SCALAR))
+      .setId(OfferID.newBuilder().setValue("o1").build())
+      .setFrameworkId(FrameworkID.newBuilder().setValue("f1").build())
+      .setSlaveId(SlaveID.newBuilder().setValue("s1").build())
+      .setHostname("host1")
+      .build()
+
+    val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]])
+
+    when(
+      driver.launchTasks(
+        Matchers.eq(Collections.singleton(offer.getId)),
+        capture.capture())
+    ).thenReturn(Status.valueOf(1))
+
+    scheduler.resourceOffers(driver, Collections.singletonList(offer))
+
+    val taskInfos = capture.getValue
+    assert(taskInfos.size() == 1)
+    val taskInfo = taskInfos.iterator().next()
+    val resources = taskInfo.getResourcesList
+    assert(scheduler.getResource(resources, "cpus") == 1.5)
+    assert(scheduler.getResource(resources, "mem") == 1200)
+    val resourcesSeq: Seq[Resource] = resources.asScala
+    val cpus = resourcesSeq.filter(_.getName.equals("cpus")).toList
+    assert(cpus.size == 2)
+    assert(cpus.exists(_.getRole().equals("role2")))
+    assert(cpus.exists(_.getRole().equals("*")))
+    val mem = resourcesSeq.filter(_.getName.equals("mem")).toList
+    assert(mem.size == 2)
+    assert(mem.exists(_.getRole().equals("role2")))
+    assert(mem.exists(_.getRole().equals("*")))
+
+    verify(driver, times(1)).launchTasks(
+      Matchers.eq(Collections.singleton(offer.getId)),
+      capture.capture()
+    )
+  }
+
+  test("escapes commandline args for the shell") {
+    setScheduler()
+
+    val conf = new SparkConf()
+    conf.setMaster("mesos://localhost:5050")
+    conf.setAppName("spark mesos")
+    val scheduler = new MesosClusterScheduler(
+      new BlackHoleMesosClusterPersistenceEngineFactory, conf) {
+      override def start(): Unit = { ready = true }
+    }
+    val escape = scheduler.shellEscape _
+    def wrapped(str: String): String = "\"" + str + "\""
+
+    // Wrapped in quotes
+    assert(escape("'should be left untouched'") === "'should be left untouched'")
+    assert(escape("\"should be left untouched\"") === "\"should be left untouched\"")
+
+    // Harmless
+    assert(escape("") === "")
+    assert(escape("harmless") === "harmless")
+    assert(escape("har-m.l3ss") === "har-m.l3ss")
+
+    // Special Chars escape
+    assert(escape("should escape this \" quote") === wrapped("should escape this \\\" quote"))
+    assert(escape("shouldescape\"quote") === wrapped("shouldescape\\\"quote"))
+    assert(escape("should escape this $ dollar") === wrapped("should escape this \\$ dollar"))
+    assert(escape("should escape this ` backtick") === wrapped("should escape this \\` backtick"))
+    assert(escape("""should escape this \ backslash""")
+      === wrapped("""should escape this \\ backslash"""))
+    assert(escape("""\"?""") === wrapped("""\\\"?"""))
+
+
+    // Special Chars no escape only wrap
+    List(" ", "'", "<", ">", "&", "|", "?", "*", ";", "!", "#", "(", ")").foreach(char => {
+      assert(escape(s"onlywrap${char}this") === wrapped(s"onlywrap${char}this"))
+    })
+  }
+
+  test("supports spark.mesos.driverEnv.*") {
+    setScheduler()
+
+    val mem = 1000
+    val cpu = 1
+
+    val response = scheduler.submitDriver(
+      new MesosDriverDescription("d1", "jar", mem, cpu, true,
+        command,
+        Map("spark.mesos.executor.home" -> "test",
+          "spark.app.name" -> "test",
+          "spark.mesos.driverEnv.TEST_ENV" -> "TEST_VAL"),
+        "s1",
+        new Date()))
+    assert(response.success)
+
+    val offer = Utils.createOffer("o1", "s1", mem, cpu)
+    scheduler.resourceOffers(driver, List(offer).asJava)
+    val tasks = Utils.verifyTaskLaunched(driver, "o1")
+    val env = tasks.head.getCommand.getEnvironment.getVariablesList.asScala.map(v =>
+      (v.getName, v.getValue)).toMap
+    assert(env.getOrElse("TEST_ENV", null) == "TEST_VAL")
+  }
+
+  test("supports spark.mesos.network.name") {
+    setScheduler()
+
+    val mem = 1000
+    val cpu = 1
+
+    val response = scheduler.submitDriver(
+      new MesosDriverDescription("d1", "jar", mem, cpu, true,
+        command,
+        Map("spark.mesos.executor.home" -> "test",
+          "spark.app.name" -> "test",
+          "spark.mesos.network.name" -> "test-network-name"),
+        "s1",
+        new Date()))
+
+    assert(response.success)
+
+    val offer = Utils.createOffer("o1", "s1", mem, cpu)
+    scheduler.resourceOffers(driver, List(offer).asJava)
+
+    val launchedTasks = Utils.verifyTaskLaunched(driver, "o1")
+    val networkInfos = launchedTasks.head.getContainer.getNetworkInfosList
+    assert(networkInfos.size == 1)
+    assert(networkInfos.get(0).getName == "test-network-name")
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala
new file mode 100644
index 0000000..a674da4
--- /dev/null
+++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala
@@ -0,0 +1,601 @@
+/*
+ * 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.mesos
+
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.duration._
+import scala.concurrent.Promise
+import scala.reflect.ClassTag
+
+import org.apache.mesos.{Protos, Scheduler, SchedulerDriver}
+import org.apache.mesos.Protos._
+import org.mockito.Matchers
+import org.mockito.Matchers._
+import org.mockito.Mockito._
+import org.scalatest.concurrent.ScalaFutures
+import org.scalatest.mock.MockitoSugar
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.{LocalSparkContext, SecurityManager, SparkConf, SparkContext, SparkFunSuite}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient
+import org.apache.spark.rpc.RpcEndpointRef
+import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor
+import org.apache.spark.scheduler.TaskSchedulerImpl
+import org.apache.spark.scheduler.cluster.mesos.Utils._
+
+class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite
+    with LocalSparkContext
+    with MockitoSugar
+    with BeforeAndAfter
+    with ScalaFutures {
+
+  private var sparkConf: SparkConf = _
+  private var driver: SchedulerDriver = _
+  private var taskScheduler: TaskSchedulerImpl = _
+  private var backend: MesosCoarseGrainedSchedulerBackend = _
+  private var externalShuffleClient: MesosExternalShuffleClient = _
+  private var driverEndpoint: RpcEndpointRef = _
+  @volatile private var stopCalled = false
+
+  // All 'requests' to the scheduler run immediately on the same thread, so
+  // demand that all futures have their value available immediately.
+  implicit override val patienceConfig = PatienceConfig(timeout = Duration(0, TimeUnit.SECONDS))
+
+  test("mesos supports killing and limiting executors") {
+    setBackend()
+    sparkConf.set("spark.driver.host", "driverHost")
+    sparkConf.set("spark.driver.port", "1234")
+
+    val minMem = backend.executorMemory(sc)
+    val minCpu = 4
+    val offers = List(Resources(minMem, minCpu))
+
+    // launches a task on a valid offer
+    offerResources(offers)
+    verifyTaskLaunched(driver, "o1")
+
+    // kills executors
+    assert(backend.doRequestTotalExecutors(0).futureValue)
+    assert(backend.doKillExecutors(Seq("0")).futureValue)
+    val taskID0 = createTaskId("0")
+    verify(driver, times(1)).killTask(taskID0)
+
+    // doesn't launch a new task when requested executors == 0
+    offerResources(offers, 2)
+    verifyDeclinedOffer(driver, createOfferId("o2"))
+
+    // Launches a new task when requested executors is positive
+    backend.doRequestTotalExecutors(2)
+    offerResources(offers, 2)
+    verifyTaskLaunched(driver, "o2")
+  }
+
+  test("mesos supports killing and relaunching tasks with executors") {
+    setBackend()
+
+    // launches a task on a valid offer
+    val minMem = backend.executorMemory(sc) + 1024
+    val minCpu = 4
+    val offer1 = Resources(minMem, minCpu)
+    val offer2 = Resources(minMem, 1)
+    offerResources(List(offer1, offer2))
+    verifyTaskLaunched(driver, "o1")
+
+    // accounts for a killed task
+    val status = createTaskStatus("0", "s1", TaskState.TASK_KILLED)
+    backend.statusUpdate(driver, status)
+    verify(driver, times(1)).reviveOffers()
+
+    // Launches a new task on a valid offer from the same slave
+    offerResources(List(offer2))
+    verifyTaskLaunched(driver, "o2")
+  }
+
+  test("mesos supports spark.executor.cores") {
+    val executorCores = 4
+    setBackend(Map("spark.executor.cores" -> executorCores.toString))
+
+    val executorMemory = backend.executorMemory(sc)
+    val offers = List(Resources(executorMemory * 2, executorCores + 1))
+    offerResources(offers)
+
+    val taskInfos = verifyTaskLaunched(driver, "o1")
+    assert(taskInfos.length == 1)
+
+    val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus")
+    assert(cpus == executorCores)
+  }
+
+  test("mesos supports unset spark.executor.cores") {
+    setBackend()
+
+    val executorMemory = backend.executorMemory(sc)
+    val offerCores = 10
+    offerResources(List(Resources(executorMemory * 2, offerCores)))
+
+    val taskInfos = verifyTaskLaunched(driver, "o1")
+    assert(taskInfos.length == 1)
+
+    val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus")
+    assert(cpus == offerCores)
+  }
+
+  test("mesos does not acquire more than spark.cores.max") {
+    val maxCores = 10
+    setBackend(Map("spark.cores.max" -> maxCores.toString))
+
+    val executorMemory = backend.executorMemory(sc)
+    offerResources(List(Resources(executorMemory, maxCores + 1)))
+
+    val taskInfos = verifyTaskLaunched(driver, "o1")
+    assert(taskInfos.length == 1)
+
+    val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus")
+    assert(cpus == maxCores)
+  }
+
+  test("mesos does not acquire gpus if not specified") {
+    setBackend()
+
+    val executorMemory = backend.executorMemory(sc)
+    offerResources(List(Resources(executorMemory, 1, 1)))
+
+    val taskInfos = verifyTaskLaunched(driver, "o1")
+    assert(taskInfos.length == 1)
+
+    val gpus = backend.getResource(taskInfos.head.getResourcesList, "gpus")
+    assert(gpus == 0.0)
+  }
+
+
+  test("mesos does not acquire more than spark.mesos.gpus.max") {
+    val maxGpus = 5
+    setBackend(Map("spark.mesos.gpus.max" -> maxGpus.toString))
+
+    val executorMemory = backend.executorMemory(sc)
+    offerResources(List(Resources(executorMemory, 1, maxGpus + 1)))
+
+    val taskInfos = verifyTaskLaunched(driver, "o1")
+    assert(taskInfos.length == 1)
+
+    val gpus = backend.getResource(taskInfos.head.getResourcesList, "gpus")
+    assert(gpus == maxGpus)
+  }
+
+
+  test("mesos declines offers that violate attribute constraints") {
+    setBackend(Map("spark.mesos.constraints" -> "x:true"))
+    offerResources(List(Resources(backend.executorMemory(sc), 4)))
+    verifyDeclinedOffer(driver, createOfferId("o1"), true)
+  }
+
+  test("mesos declines offers with a filter when reached spark.cores.max") {
+    val maxCores = 3
+    setBackend(Map("spark.cores.max" -> maxCores.toString))
+
+    val executorMemory = backend.executorMemory(sc)
+    offerResources(List(
+      Resources(executorMemory, maxCores + 1),
+      Resources(executorMemory, maxCores + 1)))
+
+    verifyTaskLaunched(driver, "o1")
+    verifyDeclinedOffer(driver, createOfferId("o2"), true)
+  }
+
+  test("mesos assigns tasks round-robin on offers") {
+    val executorCores = 4
+    val maxCores = executorCores * 2
+    setBackend(Map("spark.executor.cores" -> executorCores.toString,
+      "spark.cores.max" -> maxCores.toString))
+
+    val executorMemory = backend.executorMemory(sc)
+    offerResources(List(
+      Resources(executorMemory * 2, executorCores * 2),
+      Resources(executorMemory * 2, executorCores * 2)))
+
+    verifyTaskLaunched(driver, "o1")
+    verifyTaskLaunched(driver, "o2")
+  }
+
+  test("mesos creates multiple executors on a single slave") {
+    val executorCores = 4
+    setBackend(Map("spark.executor.cores" -> executorCores.toString))
+
+    // offer with room for two executors
+    val executorMemory = backend.executorMemory(sc)
+    offerResources(List(Resources(executorMemory * 2, executorCores * 2)))
+
+    // verify two executors were started on a single offer
+    val taskInfos = verifyTaskLaunched(driver, "o1")
+    assert(taskInfos.length == 2)
+  }
+
+  test("mesos doesn't register twice with the same shuffle service") {
+    setBackend(Map("spark.shuffle.service.enabled" -> "true"))
+    val (mem, cpu) = (backend.executorMemory(sc), 4)
+
+    val offer1 = createOffer("o1", "s1", mem, cpu)
+    backend.resourceOffers(driver, List(offer1).asJava)
+    verifyTaskLaunched(driver, "o1")
+
+    val offer2 = createOffer("o2", "s1", mem, cpu)
+    backend.resourceOffers(driver, List(offer2).asJava)
+    verifyTaskLaunched(driver, "o2")
+
+    val status1 = createTaskStatus("0", "s1", TaskState.TASK_RUNNING)
+    backend.statusUpdate(driver, status1)
+
+    val status2 = createTaskStatus("1", "s1", TaskState.TASK_RUNNING)
+    backend.statusUpdate(driver, status2)
+    verify(externalShuffleClient, times(1))
+      .registerDriverWithShuffleService(anyString, anyInt, anyLong, anyLong)
+  }
+
+  test("Port offer decline when there is no appropriate range") {
+    setBackend(Map(BLOCK_MANAGER_PORT.key -> "30100"))
+    val offeredPorts = (31100L, 31200L)
+    val (mem, cpu) = (backend.executorMemory(sc), 4)
+
+    val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts))
+    backend.resourceOffers(driver, List(offer1).asJava)
+    verify(driver, times(1)).declineOffer(offer1.getId)
+  }
+
+  test("Port offer accepted when ephemeral ports are used") {
+    setBackend()
+    val offeredPorts = (31100L, 31200L)
+    val (mem, cpu) = (backend.executorMemory(sc), 4)
+
+    val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts))
+    backend.resourceOffers(driver, List(offer1).asJava)
+    verifyTaskLaunched(driver, "o1")
+  }
+
+  test("Port offer accepted with user defined port numbers") {
+    val port = 30100
+    setBackend(Map(BLOCK_MANAGER_PORT.key -> s"$port"))
+    val offeredPorts = (30000L, 31000L)
+    val (mem, cpu) = (backend.executorMemory(sc), 4)
+
+    val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts))
+    backend.resourceOffers(driver, List(offer1).asJava)
+    val taskInfo = verifyTaskLaunched(driver, "o1")
+
+    val taskPortResources = taskInfo.head.getResourcesList.asScala.
+    find(r => r.getType == Value.Type.RANGES && r.getName == "ports")
+
+    val isPortInOffer = (r: Resource) => {
+      r.getRanges().getRangeList
+        .asScala.exists(range => range.getBegin == port && range.getEnd == port)
+    }
+    assert(taskPortResources.exists(isPortInOffer))
+  }
+
+  test("mesos kills an executor when told") {
+    setBackend()
+
+    val (mem, cpu) = (backend.executorMemory(sc), 4)
+
+    val offer1 = createOffer("o1", "s1", mem, cpu)
+    backend.resourceOffers(driver, List(offer1).asJava)
+    verifyTaskLaunched(driver, "o1")
+
+    backend.doKillExecutors(List("0"))
+    verify(driver, times(1)).killTask(createTaskId("0"))
+  }
+
+  test("weburi is set in created scheduler driver") {
+    setBackend()
+    val taskScheduler = mock[TaskSchedulerImpl]
+    when(taskScheduler.sc).thenReturn(sc)
+    val driver = mock[SchedulerDriver]
+    when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING)
+    val securityManager = mock[SecurityManager]
+
+    val backend = new MesosCoarseGrainedSchedulerBackend(
+        taskScheduler, sc, "master", securityManager) {
+      override protected def createSchedulerDriver(
+        masterUrl: String,
+        scheduler: Scheduler,
+        sparkUser: String,
+        appName: String,
+        conf: SparkConf,
+        webuiUrl: Option[String] = None,
+        checkpoint: Option[Boolean] = None,
+        failoverTimeout: Option[Double] = None,
+        frameworkId: Option[String] = None): SchedulerDriver = {
+        markRegistered()
+        assert(webuiUrl.isDefined)
+        assert(webuiUrl.get.equals("http://webui"))
+        driver
+      }
+    }
+
+    backend.start()
+  }
+
+  test("honors unset spark.mesos.containerizer") {
+    setBackend(Map("spark.mesos.executor.docker.image" -> "test"))
+
+    val (mem, cpu) = (backend.executorMemory(sc), 4)
+
+    val offer1 = createOffer("o1", "s1", mem, cpu)
+    backend.resourceOffers(driver, List(offer1).asJava)
+
+    val taskInfos = verifyTaskLaunched(driver, "o1")
+    assert(taskInfos.head.getContainer.getType == ContainerInfo.Type.DOCKER)
+  }
+
+  test("honors spark.mesos.containerizer=\"mesos\"") {
+    setBackend(Map(
+      "spark.mesos.executor.docker.image" -> "test",
+      "spark.mesos.containerizer" -> "mesos"))
+
+    val (mem, cpu) = (backend.executorMemory(sc), 4)
+
+    val offer1 = createOffer("o1", "s1", mem, cpu)
+    backend.resourceOffers(driver, List(offer1).asJava)
+
+    val taskInfos = verifyTaskLaunched(driver, "o1")
+    assert(taskInfos.head.getContainer.getType == ContainerInfo.Type.MESOS)
+  }
+
+  test("docker settings are reflected in created tasks") {
+    setBackend(Map(
+      "spark.mesos.executor.docker.image" -> "some_image",
+      "spark.mesos.executor.docker.forcePullImage" -> "true",
+      "spark.mesos.executor.docker.volumes" -> "/host_vol:/container_vol:ro",
+      "spark.mesos.executor.docker.portmaps" -> "8080:80:tcp"
+    ))
+
+    val (mem, cpu) = (backend.executorMemory(sc), 4)
+
+    val offer1 = createOffer("o1", "s1", mem, cpu)
+    backend.resourceOffers(driver, List(offer1).asJava)
+
+    val launchedTasks = verifyTaskLaunched(driver, "o1")
+    assert(launchedTasks.size == 1)
+
+    val containerInfo = launchedTasks.head.getContainer
+    assert(containerInfo.getType == ContainerInfo.Type.DOCKER)
+
+    val volumes = containerInfo.getVolumesList.asScala
+    assert(volumes.size == 1)
+
+    val volume = volumes.head
+    assert(volume.getHostPath == "/host_vol")
+    assert(volume.getContainerPath == "/container_vol")
+    assert(volume.getMode == Volume.Mode.RO)
+
+    val dockerInfo = containerInfo.getDocker
+
+    val portMappings = dockerInfo.getPortMappingsList.asScala
+    assert(portMappings.size == 1)
+
+    val portMapping = portMappings.head
+    assert(portMapping.getHostPort == 8080)
+    assert(portMapping.getContainerPort == 80)
+    assert(portMapping.getProtocol == "tcp")
+  }
+
+  test("force-pull-image option is disabled by default") {
+    setBackend(Map(
+      "spark.mesos.executor.docker.image" -> "some_image"
+    ))
+
+    val (mem, cpu) = (backend.executorMemory(sc), 4)
+
+    val offer1 = createOffer("o1", "s1", mem, cpu)
+    backend.resourceOffers(driver, List(offer1).asJava)
+
+    val launchedTasks = verifyTaskLaunched(driver, "o1")
+    assert(launchedTasks.size == 1)
+
+    val containerInfo = launchedTasks.head.getContainer
+    assert(containerInfo.getType == ContainerInfo.Type.DOCKER)
+
+    val dockerInfo = containerInfo.getDocker
+
+    assert(dockerInfo.getImage == "some_image")
+    assert(!dockerInfo.getForcePullImage)
+  }
+
+  test("Do not call removeExecutor() after backend is stopped") {
+    setBackend()
+
+    // launches a task on a valid offer
+    val offers = List(Resources(backend.executorMemory(sc), 1))
+    offerResources(offers)
+    verifyTaskLaunched(driver, "o1")
+
+    // launches a thread simulating status update
+    val statusUpdateThread = new Thread {
+      override def run(): Unit = {
+        while (!stopCalled) {
+          Thread.sleep(100)
+        }
+
+        val status = createTaskStatus("0", "s1", TaskState.TASK_FINISHED)
+        backend.statusUpdate(driver, status)
+      }
+    }.start
+
+    backend.stop()
+    // Any method of the backend involving sending messages to the driver endpoint should not
+    // be called after the backend is stopped.
+    verify(driverEndpoint, never()).askWithRetry(isA(classOf[RemoveExecutor]))(any[ClassTag[_]])
+  }
+
+  test("mesos supports spark.executor.uri") {
+    val url = "spark.spark.spark.com"
+    setBackend(Map(
+      "spark.executor.uri" -> url
+    ), false)
+
+    val (mem, cpu) = (backend.executorMemory(sc), 4)
+
+    val offer1 = createOffer("o1", "s1", mem, cpu)
+    backend.resourceOffers(driver, List(offer1).asJava)
+
+    val launchedTasks = verifyTaskLaunched(driver, "o1")
+    assert(launchedTasks.head.getCommand.getUrisList.asScala(0).getValue == url)
+  }
+
+  test("mesos supports setting fetcher cache") {
+    val url = "spark.spark.spark.com"
+    setBackend(Map(
+      "spark.mesos.fetcherCache.enable" -> "true",
+      "spark.executor.uri" -> url
+    ), false)
+    val offers = List(Resources(backend.executorMemory(sc), 1))
+    offerResources(offers)
+    val launchedTasks = verifyTaskLaunched(driver, "o1")
+    val uris = launchedTasks.head.getCommand.getUrisList
+    assert(uris.size() == 1)
+    assert(uris.asScala.head.getCache)
+  }
+
+  test("mesos supports disabling fetcher cache") {
+    val url = "spark.spark.spark.com"
+    setBackend(Map(
+      "spark.mesos.fetcherCache.enable" -> "false",
+      "spark.executor.uri" -> url
+    ), false)
+    val offers = List(Resources(backend.executorMemory(sc), 1))
+    offerResources(offers)
+    val launchedTasks = verifyTaskLaunched(driver, "o1")
+    val uris = launchedTasks.head.getCommand.getUrisList
+    assert(uris.size() == 1)
+    assert(!uris.asScala.head.getCache)
+  }
+
+  test("mesos supports spark.mesos.network.name") {
+    setBackend(Map(
+      "spark.mesos.network.name" -> "test-network-name"
+    ))
+
+    val (mem, cpu) = (backend.executorMemory(sc), 4)
+
+    val offer1 = createOffer("o1", "s1", mem, cpu)
+    backend.resourceOffers(driver, List(offer1).asJava)
+
+    val launchedTasks = verifyTaskLaunched(driver, "o1")
+    val networkInfos = launchedTasks.head.getContainer.getNetworkInfosList
+    assert(networkInfos.size == 1)
+    assert(networkInfos.get(0).getName == "test-network-name")
+  }
+
+  private case class Resources(mem: Int, cpus: Int, gpus: Int = 0)
+
+  private def verifyDeclinedOffer(driver: SchedulerDriver,
+      offerId: OfferID,
+      filter: Boolean = false): Unit = {
+    if (filter) {
+      verify(driver, times(1)).declineOffer(Matchers.eq(offerId), anyObject[Filters])
+    } else {
+      verify(driver, times(1)).declineOffer(Matchers.eq(offerId))
+    }
+  }
+
+  private def offerResources(offers: List[Resources], startId: Int = 1): Unit = {
+    val mesosOffers = offers.zipWithIndex.map {case (offer, i) =>
+      createOffer(s"o${i + startId}", s"s${i + startId}", offer.mem, offer.cpus, None, offer.gpus)}
+
+    backend.resourceOffers(driver, mesosOffers.asJava)
+  }
+
+  private def createTaskStatus(taskId: String, slaveId: String, state: TaskState): TaskStatus = {
+    TaskStatus.newBuilder()
+      .setTaskId(TaskID.newBuilder().setValue(taskId).build())
+      .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build())
+      .setState(state)
+      .build
+  }
+
+  private def createSchedulerBackend(
+      taskScheduler: TaskSchedulerImpl,
+      driver: SchedulerDriver,
+      shuffleClient: MesosExternalShuffleClient,
+      endpoint: RpcEndpointRef): MesosCoarseGrainedSchedulerBackend = {
+    val securityManager = mock[SecurityManager]
+
+    val backend = new MesosCoarseGrainedSchedulerBackend(
+        taskScheduler, sc, "master", securityManager) {
+      override protected def createSchedulerDriver(
+          masterUrl: String,
+          scheduler: Scheduler,
+          sparkUser: String,
+          appName: String,
+          conf: SparkConf,
+          webuiUrl: Option[String] = None,
+          checkpoint: Option[Boolean] = None,
+          failoverTimeout: Option[Double] = None,
+          frameworkId: Option[String] = None): SchedulerDriver = driver
+
+      override protected def getShuffleClient(): MesosExternalShuffleClient = shuffleClient
+
+      override protected def createDriverEndpointRef(
+          properties: ArrayBuffer[(String, String)]): RpcEndpointRef = endpoint
+
+      // override to avoid race condition with the driver thread on `mesosDriver`
+      override def startScheduler(newDriver: SchedulerDriver): Unit = {
+        mesosDriver = newDriver
+      }
+
+      override def stopExecutors(): Unit = {
+        stopCalled = true
+      }
+
+      markRegistered()
+    }
+    backend.start()
+    backend
+  }
+
+  private def setBackend(sparkConfVars: Map[String, String] = null,
+      setHome: Boolean = true) {
+    sparkConf = (new SparkConf)
+      .setMaster("local[*]")
+      .setAppName("test-mesos-dynamic-alloc")
+      .set("spark.mesos.driver.webui.url", "http://webui")
+
+    if (setHome) {
+      sparkConf.setSparkHome("/path")
+    }
+
+    if (sparkConfVars != null) {
+      sparkConf.setAll(sparkConfVars)
+    }
+
+    sc = new SparkContext(sparkConf)
+
+    driver = mock[SchedulerDriver]
+    when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING)
+    taskScheduler = mock[TaskSchedulerImpl]
+    when(taskScheduler.sc).thenReturn(sc)
+    externalShuffleClient = mock[MesosExternalShuffleClient]
+    driverEndpoint = mock[RpcEndpointRef]
+    when(driverEndpoint.ask(any())(any())).thenReturn(Promise().future)
+
+    backend = createSchedulerBackend(taskScheduler, driver, externalShuffleClient, driverEndpoint)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/81e5619c/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala
new file mode 100644
index 0000000..1d7a86f
--- /dev/null
+++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala
@@ -0,0 +1,385 @@
+/*
+ * 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.mesos
+
+import java.nio.ByteBuffer
+import java.util.Arrays
+import java.util.Collection
+import java.util.Collections
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.mesos.{Protos, Scheduler, SchedulerDriver}
+import org.apache.mesos.Protos._
+import org.apache.mesos.Protos.Value.Scalar
+import org.mockito.{ArgumentCaptor, Matchers}
+import org.mockito.Matchers._
+import org.mockito.Mockito._
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite}
+import org.apache.spark.executor.MesosExecutorBackend
+import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded,
+  TaskDescription, TaskSchedulerImpl, WorkerOffer}
+import org.apache.spark.scheduler.cluster.ExecutorInfo
+
+class MesosFineGrainedSchedulerBackendSuite
+  extends SparkFunSuite with LocalSparkContext with MockitoSugar {
+
+  test("weburi is set in created scheduler driver") {
+    val conf = new SparkConf
+    conf.set("spark.mesos.driver.webui.url", "http://webui")
+    conf.set("spark.app.name", "name1")
+
+    val sc = mock[SparkContext]
+    when(sc.conf).thenReturn(conf)
+    when(sc.sparkUser).thenReturn("sparkUser1")
+    when(sc.appName).thenReturn("appName1")
+
+    val taskScheduler = mock[TaskSchedulerImpl]
+    val driver = mock[SchedulerDriver]
+    when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING)
+
+    val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") {
+      override protected def createSchedulerDriver(
+        masterUrl: String,
+        scheduler: Scheduler,
+        sparkUser: String,
+        appName: String,
+        conf: SparkConf,
+        webuiUrl: Option[String] = None,
+        checkpoint: Option[Boolean] = None,
+        failoverTimeout: Option[Double] = None,
+        frameworkId: Option[String] = None): SchedulerDriver = {
+        markRegistered()
+        assert(webuiUrl.isDefined)
+        assert(webuiUrl.get.equals("http://webui"))
+        driver
+      }
+    }
+
+    backend.start()
+  }
+
+  test("Use configured mesosExecutor.cores for ExecutorInfo") {
+    val mesosExecutorCores = 3
+    val conf = new SparkConf
+    conf.set("spark.mesos.mesosExecutor.cores", mesosExecutorCores.toString)
+
+    val listenerBus = mock[LiveListenerBus]
+    listenerBus.post(
+      SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty)))
+
+    val sc = mock[SparkContext]
+    when(sc.getSparkHome()).thenReturn(Option("/spark-home"))
+
+    when(sc.conf).thenReturn(conf)
+    when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String])
+    when(sc.executorMemory).thenReturn(100)
+    when(sc.listenerBus).thenReturn(listenerBus)
+    val taskScheduler = mock[TaskSchedulerImpl]
+    when(taskScheduler.CPUS_PER_TASK).thenReturn(2)
+
+    val mesosSchedulerBackend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master")
+
+    val resources = Arrays.asList(
+      mesosSchedulerBackend.createResource("cpus", 4),
+      mesosSchedulerBackend.createResource("mem", 1024))
+    // uri is null.
+    val (executorInfo, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id")
+    val executorResources = executorInfo.getResourcesList
+    val cpus = executorResources.asScala.find(_.getName.equals("cpus")).get.getScalar.getValue
+
+    assert(cpus === mesosExecutorCores)
+  }
+
+  test("check spark-class location correctly") {
+    val conf = new SparkConf
+    conf.set("spark.mesos.executor.home", "/mesos-home")
+
+    val listenerBus = mock[LiveListenerBus]
+    listenerBus.post(
+      SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty)))
+
+    val sc = mock[SparkContext]
+    when(sc.getSparkHome()).thenReturn(Option("/spark-home"))
+
+    when(sc.conf).thenReturn(conf)
+    when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String])
+    when(sc.executorMemory).thenReturn(100)
+    when(sc.listenerBus).thenReturn(listenerBus)
+    val taskScheduler = mock[TaskSchedulerImpl]
+    when(taskScheduler.CPUS_PER_TASK).thenReturn(2)
+
+    val mesosSchedulerBackend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master")
+
+    val resources = Arrays.asList(
+      mesosSchedulerBackend.createResource("cpus", 4),
+      mesosSchedulerBackend.createResource("mem", 1024))
+    // uri is null.
+    val (executorInfo, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id")
+    assert(executorInfo.getCommand.getValue ===
+      s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}")
+
+    // uri exists.
+    conf.set("spark.executor.uri", "hdfs:///test-app-1.0.0.tgz")
+    val (executorInfo1, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id")
+    assert(executorInfo1.getCommand.getValue ===
+      s"cd test-app-1*;  ./bin/spark-class ${classOf[MesosExecutorBackend].getName}")
+  }
+
+  test("spark docker properties correctly populate the DockerInfo message") {
+    val taskScheduler = mock[TaskSchedulerImpl]
+
+    val conf = new SparkConf()
+      .set("spark.mesos.executor.docker.image", "spark/mock")
+      .set("spark.mesos.executor.docker.forcePullImage", "true")
+      .set("spark.mesos.executor.docker.volumes", "/a,/b:/b,/c:/c:rw,/d:ro,/e:/e:ro")
+      .set("spark.mesos.executor.docker.portmaps", "80:8080,53:53:tcp")
+
+    val listenerBus = mock[LiveListenerBus]
+    listenerBus.post(
+      SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty)))
+
+    val sc = mock[SparkContext]
+    when(sc.executorMemory).thenReturn(100)
+    when(sc.getSparkHome()).thenReturn(Option("/spark-home"))
+    when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String])
+    when(sc.conf).thenReturn(conf)
+    when(sc.listenerBus).thenReturn(listenerBus)
+
+    val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master")
+
+    val (execInfo, _) = backend.createExecutorInfo(
+      Arrays.asList(backend.createResource("cpus", 4)), "mockExecutor")
+    assert(execInfo.getContainer.getDocker.getImage.equals("spark/mock"))
+    assert(execInfo.getContainer.getDocker.getForcePullImage.equals(true))
+    val portmaps = execInfo.getContainer.getDocker.getPortMappingsList
+    assert(portmaps.get(0).getHostPort.equals(80))
+    assert(portmaps.get(0).getContainerPort.equals(8080))
+    assert(portmaps.get(0).getProtocol.equals("tcp"))
+    assert(portmaps.get(1).getHostPort.equals(53))
+    assert(portmaps.get(1).getContainerPort.equals(53))
+    assert(portmaps.get(1).getProtocol.equals("tcp"))
+    val volumes = execInfo.getContainer.getVolumesList
+    assert(volumes.get(0).getContainerPath.equals("/a"))
+    assert(volumes.get(0).getMode.equals(Volume.Mode.RW))
+    assert(volumes.get(1).getContainerPath.equals("/b"))
+    assert(volumes.get(1).getHostPath.equals("/b"))
+    assert(volumes.get(1).getMode.equals(Volume.Mode.RW))
+    assert(volumes.get(2).getContainerPath.equals("/c"))
+    assert(volumes.get(2).getHostPath.equals("/c"))
+    assert(volumes.get(2).getMode.equals(Volume.Mode.RW))
+    assert(volumes.get(3).getContainerPath.equals("/d"))
+    assert(volumes.get(3).getMode.equals(Volume.Mode.RO))
+    assert(volumes.get(4).getContainerPath.equals("/e"))
+    assert(volumes.get(4).getHostPath.equals("/e"))
+    assert(volumes.get(4).getMode.equals(Volume.Mode.RO))
+  }
+
+  test("mesos resource offers result in launching tasks") {
+    def createOffer(id: Int, mem: Int, cpu: Int): Offer = {
+      val builder = Offer.newBuilder()
+      builder.addResourcesBuilder()
+        .setName("mem")
+        .setType(Value.Type.SCALAR)
+        .setScalar(Scalar.newBuilder().setValue(mem))
+      builder.addResourcesBuilder()
+        .setName("cpus")
+        .setType(Value.Type.SCALAR)
+        .setScalar(Scalar.newBuilder().setValue(cpu))
+      builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build())
+        .setFrameworkId(FrameworkID.newBuilder().setValue("f1"))
+        .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}"))
+        .setHostname(s"host${id.toString}").build()
+    }
+
+    val driver = mock[SchedulerDriver]
+    val taskScheduler = mock[TaskSchedulerImpl]
+
+    val listenerBus = mock[LiveListenerBus]
+    listenerBus.post(
+      SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty)))
+
+    val sc = mock[SparkContext]
+    when(sc.executorMemory).thenReturn(100)
+    when(sc.getSparkHome()).thenReturn(Option("/path"))
+    when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String])
+    when(sc.conf).thenReturn(new SparkConf)
+    when(sc.listenerBus).thenReturn(listenerBus)
+
+    val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master")
+
+    val minMem = backend.executorMemory(sc)
+    val minCpu = 4
+
+    val mesosOffers = new java.util.ArrayList[Offer]
+    mesosOffers.add(createOffer(1, minMem, minCpu))
+    mesosOffers.add(createOffer(2, minMem - 1, minCpu))
+    mesosOffers.add(createOffer(3, minMem, minCpu))
+
+    val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2)
+    expectedWorkerOffers += new WorkerOffer(
+      mesosOffers.get(0).getSlaveId.getValue,
+      mesosOffers.get(0).getHostname,
+      (minCpu - backend.mesosExecutorCores).toInt
+    )
+    expectedWorkerOffers += new WorkerOffer(
+      mesosOffers.get(2).getSlaveId.getValue,
+      mesosOffers.get(2).getHostname,
+      (minCpu - backend.mesosExecutorCores).toInt
+    )
+    val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0)))
+    when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc)))
+    when(taskScheduler.CPUS_PER_TASK).thenReturn(2)
+
+    val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]])
+    when(
+      driver.launchTasks(
+        Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)),
+        capture.capture(),
+        any(classOf[Filters])
+      )
+    ).thenReturn(Status.valueOf(1))
+    when(driver.declineOffer(mesosOffers.get(1).getId)).thenReturn(Status.valueOf(1))
+    when(driver.declineOffer(mesosOffers.get(2).getId)).thenReturn(Status.valueOf(1))
+
+    backend.resourceOffers(driver, mesosOffers)
+
+    verify(driver, times(1)).launchTasks(
+      Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)),
+      capture.capture(),
+      any(classOf[Filters])
+    )
+    verify(driver, times(1)).declineOffer(mesosOffers.get(1).getId)
+    verify(driver, times(1)).declineOffer(mesosOffers.get(2).getId)
+    assert(capture.getValue.size() === 1)
+    val taskInfo = capture.getValue.iterator().next()
+    assert(taskInfo.getName.equals("n1"))
+    val cpus = taskInfo.getResourcesList.get(0)
+    assert(cpus.getName.equals("cpus"))
+    assert(cpus.getScalar.getValue.equals(2.0))
+    assert(taskInfo.getSlaveId.getValue.equals("s1"))
+
+    // Unwanted resources offered on an existing node. Make sure they are declined
+    val mesosOffers2 = new java.util.ArrayList[Offer]
+    mesosOffers2.add(createOffer(1, minMem, minCpu))
+    reset(taskScheduler)
+    reset(driver)
+    when(taskScheduler.resourceOffers(any(classOf[IndexedSeq[WorkerOffer]]))).thenReturn(Seq(Seq()))
+    when(taskScheduler.CPUS_PER_TASK).thenReturn(2)
+    when(driver.declineOffer(mesosOffers2.get(0).getId)).thenReturn(Status.valueOf(1))
+
+    backend.resourceOffers(driver, mesosOffers2)
+    verify(driver, times(1)).declineOffer(mesosOffers2.get(0).getId)
+  }
+
+  test("can handle multiple roles") {
+    val driver = mock[SchedulerDriver]
+    val taskScheduler = mock[TaskSchedulerImpl]
+
+    val listenerBus = mock[LiveListenerBus]
+    listenerBus.post(
+      SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty)))
+
+    val sc = mock[SparkContext]
+    when(sc.executorMemory).thenReturn(100)
+    when(sc.getSparkHome()).thenReturn(Option("/path"))
+    when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String])
+    when(sc.conf).thenReturn(new SparkConf)
+    when(sc.listenerBus).thenReturn(listenerBus)
+
+    val id = 1
+    val builder = Offer.newBuilder()
+    builder.addResourcesBuilder()
+      .setName("mem")
+      .setType(Value.Type.SCALAR)
+      .setRole("prod")
+      .setScalar(Scalar.newBuilder().setValue(500))
+    builder.addResourcesBuilder()
+      .setName("cpus")
+      .setRole("prod")
+      .setType(Value.Type.SCALAR)
+      .setScalar(Scalar.newBuilder().setValue(1))
+    builder.addResourcesBuilder()
+      .setName("mem")
+      .setRole("dev")
+      .setType(Value.Type.SCALAR)
+      .setScalar(Scalar.newBuilder().setValue(600))
+    builder.addResourcesBuilder()
+      .setName("cpus")
+      .setRole("dev")
+      .setType(Value.Type.SCALAR)
+      .setScalar(Scalar.newBuilder().setValue(2))
+    val offer = builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build())
+      .setFrameworkId(FrameworkID.newBuilder().setValue("f1"))
+      .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}"))
+      .setHostname(s"host${id.toString}").build()
+
+    val mesosOffers = new java.util.ArrayList[Offer]
+    mesosOffers.add(offer)
+
+    val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master")
+
+    val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](1)
+    expectedWorkerOffers += new WorkerOffer(
+      mesosOffers.get(0).getSlaveId.getValue,
+      mesosOffers.get(0).getHostname,
+      2 // Deducting 1 for executor
+    )
+
+    val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0)))
+    when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc)))
+    when(taskScheduler.CPUS_PER_TASK).thenReturn(1)
+
+    val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]])
+    when(
+      driver.launchTasks(
+        Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)),
+        capture.capture(),
+        any(classOf[Filters])
+      )
+    ).thenReturn(Status.valueOf(1))
+
+    backend.resourceOffers(driver, mesosOffers)
+
+    verify(driver, times(1)).launchTasks(
+      Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)),
+      capture.capture(),
+      any(classOf[Filters])
+    )
+
+    assert(capture.getValue.size() === 1)
+    val taskInfo = capture.getValue.iterator().next()
+    assert(taskInfo.getName.equals("n1"))
+    assert(taskInfo.getResourcesCount === 1)
+    val cpusDev = taskInfo.getResourcesList.get(0)
+    assert(cpusDev.getName.equals("cpus"))
+    assert(cpusDev.getScalar.getValue.equals(1.0))
+    assert(cpusDev.getRole.equals("dev"))
+    val executorResources = taskInfo.getExecutor.getResourcesList.asScala
+    assert(executorResources.exists { r =>
+      r.getName.equals("mem") && r.getScalar.getValue.equals(484.0) && r.getRole.equals("prod")
+    })
+    assert(executorResources.exists { r =>
+      r.getName.equals("cpus") && r.getScalar.getValue.equals(1.0) && r.getRole.equals("prod")
+    })
+  }
+}


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