From 1ef58dadccb878f70ffd638b2d0f260c2939eda9 Mon Sep 17 00:00:00 2001
From: Matei Zaharia <matei@eecs.berkeley.edu>
Date: Mon, 25 Feb 2013 14:01:24 -0800
Subject: [PATCH] Add a config property for Akka lifecycle event logging

---
 core/src/main/scala/spark/util/AkkaUtils.scala | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)

diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala
index f7b9e2d4ce..3e805b7831 100644
--- a/core/src/main/scala/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/spark/util/AkkaUtils.scala
@@ -31,20 +31,22 @@ private[spark] object AkkaUtils {
     val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt
     val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt
     val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt
+    val lifecycleEvents = System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean
     val akkaConf = ConfigFactory.parseString("""
       akka.daemonic = on
       akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
       akka.stdout-loglevel = "ERROR"
       akka.actor.provider = "akka.remote.RemoteActorRefProvider"
       akka.remote.transport = "akka.remote.netty.NettyRemoteTransport"
-      akka.remote.log-remote-lifecycle-events = off
       akka.remote.netty.hostname = "%s"
       akka.remote.netty.port = %d
       akka.remote.netty.connection-timeout = %ds
       akka.remote.netty.message-frame-size = %d MiB
       akka.remote.netty.execution-pool-size = %d
       akka.actor.default-dispatcher.throughput = %d
-      """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize))
+      akka.remote.log-remote-lifecycle-events = %s
+      """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize,
+                 if (lifecycleEvents) "on" else "off"))
 
     val actorSystem = ActorSystem(name, akkaConf, getClass.getClassLoader)
 
-- 
GitLab