Skip to content
Snippets Groups Projects
Commit 1ef58dad authored by Matei Zaharia's avatar Matei Zaharia
Browse files

Add a config property for Akka lifecycle event logging

parent 6b87ef7c
No related branches found
No related tags found
No related merge requests found
......@@ -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)
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment