aboutsummaryrefslogtreecommitdiff
path: root/yarn/alpha
diff options
context:
space:
mode:
authorAndrew Or <andrewor14@gmail.com>2014-08-25 23:36:09 -0700
committerAndrew Or <andrewor14@gmail.com>2014-08-25 23:36:09 -0700
commitb21ae5bbb9baa966f69303a30659aa8bbb2098da (patch)
tree93b422b0e63076694ad5f5accae4fc6598b2288c /yarn/alpha
parent52fbdc2deddcdba02bf5945a36e15870021ec890 (diff)
downloadspark-b21ae5bbb9baa966f69303a30659aa8bbb2098da.tar.gz
spark-b21ae5bbb9baa966f69303a30659aa8bbb2098da.tar.bz2
spark-b21ae5bbb9baa966f69303a30659aa8bbb2098da.zip
[SPARK-2886] Use more specific actor system name than "spark"
As of #1777 we log the name of the actor system when it binds to a port. The current name "spark" is super general and does not convey any meaning. For instance, the following line is taken from my driver log after setting `spark.driver.port` to 5001. ``` 14/08/13 19:33:29 INFO Remoting: Remoting started; listening on addresses: [akka.tcp://sparkandrews-mbp:5001] 14/08/13 19:33:29 INFO Remoting: Remoting now listens on addresses: [akka.tcp://sparkandrews-mbp:5001] 14/08/06 13:40:05 INFO Utils: Successfully started service 'spark' on port 5001. ``` This commit renames this to "sparkDriver" and "sparkExecutor". The goal of this unambitious PR is simply to make the logged information more explicit without introducing any change in functionality. Author: Andrew Or <andrewor14@gmail.com> Closes #1810 from andrewor14/service-name and squashes the following commits: 8c459ed [Andrew Or] Use a common variable for driver/executor actor system names 3a92843 [Andrew Or] Change actor name to sparkDriver and sparkExecutor 921363e [Andrew Or] Merge branch 'master' of github.com:apache/spark into service-name c8c6a62 [Andrew Or] Do not include hyphens in actor name 1c1b42e [Andrew Or] Avoid spaces in akka system name f644b55 [Andrew Or] Use more specific service name
Diffstat (limited to 'yarn/alpha')
-rw-r--r--yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala9
-rw-r--r--yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala8
2 files changed, 11 insertions, 6 deletions
diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
index c3310fbc24..155dd88aa2 100644
--- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
@@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC
import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
import akka.actor._
import akka.remote._
-import org.apache.spark.{Logging, SecurityManager, SparkConf}
+import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv}
import org.apache.spark.util.{Utils, AkkaUtils}
import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter
@@ -210,8 +210,11 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp
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)
+ val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format(
+ SparkEnv.driverActorSystemName,
+ driverHost,
+ driverPort.toString,
+ CoarseGrainedSchedulerBackend.ACTOR_NAME)
actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM")
}
diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index 80e0162e9f..568a6ef932 100644
--- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -26,7 +26,7 @@ import scala.collection
import scala.collection.JavaConversions._
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
-import org.apache.spark.{Logging, SparkConf}
+import org.apache.spark.{Logging, SparkConf, SparkEnv}
import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl}
import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
import org.apache.spark.util.Utils
@@ -245,8 +245,10 @@ private[yarn] class YarnAllocationHandler(
// Deallocate + allocate can result in reusing id's wrongly - so use a different counter
// (executorIdCounter)
val executorId = executorIdCounter.incrementAndGet().toString
- val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
- sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"),
+ val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format(
+ SparkEnv.driverActorSystemName,
+ sparkConf.get("spark.driver.host"),
+ sparkConf.get("spark.driver.port"),
CoarseGrainedSchedulerBackend.ACTOR_NAME)
logInfo("launching container on " + containerId + " host " + executorHostname)