diff options
author | Raymond Liu <raymond.liu@intel.com> | 2013-12-17 13:19:14 +0800 |
---|---|---|
committer | Raymond Liu <raymond.liu@intel.com> | 2014-01-03 12:14:37 +0800 |
commit | dd6d347f4f9bee6f7509fffe07a1a2d74803f783 (patch) | |
tree | 6b49c8b5f90cb9ac5982e4ca46dbfaac6894b2e1 /yarn | |
parent | 7815a3ace97848b1d4bab4ef6823fc6d55cfb72e (diff) | |
download | spark-dd6d347f4f9bee6f7509fffe07a1a2d74803f783.tar.gz spark-dd6d347f4f9bee6f7509fffe07a1a2d74803f783.tar.bz2 spark-dd6d347f4f9bee6f7509fffe07a1a2d74803f783.zip |
A few clean up for yarn 2.0 code
Diffstat (limited to 'yarn')
-rw-r--r-- | yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala | 13 | ||||
-rw-r--r-- | yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala | 2 |
2 files changed, 7 insertions, 8 deletions
diff --git a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index a8de89c670..1ef099aceb 100644 --- a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -39,28 +39,27 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte def this(args: ApplicationMasterArguments) = this(args, new Configuration()) private val rpc: YarnRPC = YarnRPC.create(conf) - private var resourceManager: AMRMProtocol = null - private var appAttemptId: ApplicationAttemptId = null - private var reporterThread: Thread = null + private var resourceManager: AMRMProtocol = _ + private var appAttemptId: ApplicationAttemptId = _ + private var reporterThread: Thread = _ private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - private var yarnAllocator: YarnAllocationHandler = null + private var yarnAllocator: YarnAllocationHandler = _ private var driverClosed:Boolean = false private val sparkConf = new SparkConf val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf)._1 - var actor: ActorRef = null + var actor: ActorRef = _ // This actor just working as a monitor to watch on Driver Actor. class MonitorActor(driverUrl: String) extends Actor { - var driver: ActorSelection = null + var driver: ActorSelection = _ override def preStart() { logInfo("Listen to driver: " + driverUrl) driver = context.actorSelection(driverUrl) - driver ! "hello" context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) } diff --git a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala index 6a90cc51cf..4f34bd913e 100644 --- a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala @@ -51,7 +51,7 @@ class WorkerRunnable( extends Runnable with Logging { var rpc: YarnRPC = YarnRPC.create(conf) - var cm: ContainerManager = null + var cm: ContainerManager = _ val yarnConf: YarnConfiguration = new YarnConfiguration(conf) def run = { |