diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala
index f7b9e2d4ce9868f020e8c5ec340b9250175ad28b..3e805b78314c78267e86fa2c6f9a0e359637f624 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)