Updated Branches:
  refs/heads/master 5b0986a1d -> a2e7e0497

Quite akka when remote lifecycle logging is disabled.

I noticed when connecting to a standalone cluster Spark gives a bunch
of Akka ERROR logs that make it seem like something is failing.

This patch does two things:

1. Akka dead letter logging is turned on/off according to the existing
   lifecycle spark property.
2. We explicitly silence akka's EndpointWriter log in log4j. This is necessary
   because for some reason that log doesn't pick up on the lifecycle
   logging settings. After a few hours of debugging this was the only solution
   I found that worked.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/aaaa6731
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/aaaa6731
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/aaaa6731

Branch: refs/heads/master
Commit: aaaa6731845495743aff4cc9bd64a54b9aa36c27
Parents: d43ad3e
Author: Patrick Wendell <pwend...@gmail.com>
Authored: Sun Jan 5 13:57:42 2014 -0800
Committer: Patrick Wendell <pwend...@gmail.com>
Committed: Sun Jan 5 15:15:59 2014 -0800

----------------------------------------------------------------------
 .../main/scala/org/apache/spark/util/AkkaUtils.scala  | 14 ++++++++++++--
 1 file changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aaaa6731/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala 
b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
index 362cea5..5729334 100644
--- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -21,6 +21,8 @@ import scala.concurrent.duration.{Duration, FiniteDuration}
 
 import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem}
 import com.typesafe.config.ConfigFactory
+import org.apache.log4j.{Level, Logger}
+
 import org.apache.spark.SparkConf
 
 /**
@@ -47,8 +49,13 @@ private[spark] object AkkaUtils {
     val akkaTimeout = conf.get("spark.akka.timeout", "100").toInt
 
     val akkaFrameSize = conf.get("spark.akka.frameSize", "10").toInt
-    val lifecycleEvents =
-      if (conf.get("spark.akka.logLifecycleEvents", "false").toBoolean) "on" 
else "off"
+    val akkaLogLifecycleEvents = conf.get("spark.akka.logLifecycleEvents", 
"false").toBoolean
+    val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off"
+    if (!akkaLogLifecycleEvents) {
+      Option(Logger.getLogger("akka.remote.EndpointWriter")).map(l => 
l.setLevel(Level.FATAL))
+    }
+
+    val logAkkaConfig = if (conf.get("spark.akka.logAkkaConfig", 
"false").toBoolean) "on" else "off"
 
     val akkaHeartBeatPauses = conf.get("spark.akka.heartbeat.pauses", 
"600").toInt
     val akkaFailureDetector =
@@ -73,7 +80,10 @@ private[spark] object AkkaUtils {
       |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB
       |akka.remote.netty.tcp.execution-pool-size = $akkaThreads
       |akka.actor.default-dispatcher.throughput = $akkaBatchSize
+      |akka.log-config-on-start = $logAkkaConfig
       |akka.remote.log-remote-lifecycle-events = $lifecycleEvents
+      |akka.log-dead-letters = $lifecycleEvents
+      |akka.log-dead-letters-during-shutdown = $lifecycleEvents
       """.stripMargin)
 
     val actorSystem = if (indestructible) {

Reply via email to