aboutsummaryrefslogtreecommitdiff
path: root/core
diff options
context:
space:
mode:
authorPrashant Sharma <prashant.s@imaginea.com>2013-11-28 16:07:15 +0530
committerPrashant Sharma <prashant.s@imaginea.com>2013-11-29 13:41:05 +0530
commit1bc83ca79187979f58385d3f28236111217174e0 (patch)
tree72fb1511c61e60ffa25b4c8c927e32375334dbda /core
parent3ec5d7476690a5f8a011ec3eaa2d9003f9559b12 (diff)
downloadspark-1bc83ca79187979f58385d3f28236111217174e0.tar.gz
spark-1bc83ca79187979f58385d3f28236111217174e0.tar.bz2
spark-1bc83ca79187979f58385d3f28236111217174e0.zip
Changed defaults for akka to almost disable failure detector.
Diffstat (limited to 'core')
-rw-r--r--core/src/main/scala/org/apache/spark/util/AkkaUtils.scala10
1 files changed, 6 insertions, 4 deletions
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 3444d8fdfe..5df8213d74 100644
--- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -39,21 +39,22 @@ private[spark] object AkkaUtils {
val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt
val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt
- val akkaTimeout = System.getProperty("spark.akka.timeout", "60").toInt
+ val akkaTimeout = System.getProperty("spark.akka.timeout", "100").toInt
val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt
val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off"
- val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "60").toInt
+ val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "600").toInt
val akkaFailureDetector =
- System.getProperty("spark.akka.failure-detector.threshold", "12.0").toDouble
- val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "5").toInt
+ System.getProperty("spark.akka.failure-detector.threshold", "300.0").toDouble
+ val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "1000").toInt
val akkaConf = ConfigFactory.parseString(
s"""
|akka.daemonic = on
|akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
|akka.stdout-loglevel = "ERROR"
+ |akka.jvm-exit-on-fatal-error = off
|akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s
|akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s
|akka.remote.transport-failure-detector.threshold = $akkaFailureDetector
@@ -61,6 +62,7 @@ private[spark] object AkkaUtils {
|akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport"
|akka.remote.netty.tcp.hostname = "$host"
|akka.remote.netty.tcp.port = $port
+ |akka.remote.netty.tcp.tcp-nodelay = on
|akka.remote.netty.tcp.connection-timeout = $akkaTimeout s
|akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB
|akka.remote.netty.tcp.execution-pool-size = $akkaThreads