diff options
author | Thomas Graves <tgraves@apache.org> | 2014-01-02 17:11:16 -0600 |
---|---|---|
committer | Thomas Graves <tgraves@apache.org> | 2014-01-02 17:11:16 -0600 |
commit | fced7885cb6cd09761578f960540d739bcbb465a (patch) | |
tree | f1c39d9a7f40cd0833d62ee56d787221dae73f2a /yarn/src/main | |
parent | c6de982be69cd50e66375cfea3d6c3267a01583b (diff) | |
download | spark-fced7885cb6cd09761578f960540d739bcbb465a.tar.gz spark-fced7885cb6cd09761578f960540d739bcbb465a.tar.bz2 spark-fced7885cb6cd09761578f960540d739bcbb465a.zip |
fix yarn-client
Diffstat (limited to 'yarn/src/main')
-rw-r--r-- | yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala | 9 |
1 files changed, 5 insertions, 4 deletions
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 28259de68f..a8de89c670 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -46,9 +46,10 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte private var yarnAllocator: YarnAllocationHandler = null private var driverClosed:Boolean = false + private val sparkConf = new SparkConf val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, - conf = new SparkConf)._1 + conf = sparkConf)._1 var actor: ActorRef = null // This actor just working as a monitor to watch on Driver Actor. @@ -163,8 +164,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte Thread.sleep(100) } } - conf.set("spark.driver.host", driverHost) - conf.set("spark.driver.port", driverPort.toString) + sparkConf.set("spark.driver.host", driverHost) + sparkConf.set("spark.driver.port", driverPort.toString) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) @@ -180,7 +181,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte scala.collection.immutable.Map() yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, - args, preferredNodeLocationData, new SparkConf) + args, preferredNodeLocationData, sparkConf) logInfo("Allocating " + args.numWorkers + " workers.") // Wait until all containers have finished |