GitHub user thomas-gerber opened a pull request:
https://github.com/apache/spark/pull/4541
akka failure detection setting mappings fix
Based on the [akka
documentation](http://doc.akka.io/docs/akka/snapshot/general/configuration.html),
the configuration keys of interest for akka failure detection start with
*akka.remote.watch-failure-detector*, **not** with
*akka.remote.transport-failure-detector*.
I think that could explain why I experience Executors randomly loosing all
their cached partitions. When looking at the logs of such an Executor, I would
see akka errors like:
````
2015-02-11 21:33:50,761 WARN [Driver Heartbeater] util.AkkaUtils
(Logging.scala:logWarning(92)) - Error sending message in 1 attempts
java.util.concurrent.TimeoutException: Futures timed out after [30 seconds]
````
and
````
2015-02-11 21:35:14,722 WARN [Executor task launch worker-372]
util.AkkaUtils (Logging.scala:logWarning(92)) - Error sending message in 3
attempts
java.util.concurrent.TimeoutException: Futures timed out after [30 seconds]
````
and
````
2015-02-11 21:35:17,735 ERROR [Executor task launch worker-370]
executor.Executor (Logging.scala:logError(96)) - Exception in task 1476.0 in
stage 28.0 (TID 146476)
org.apache.spark.SparkException: Error sending message [message =
UpdateBlockInfo(BlockManagerId(23, ip-10-0-12-82.ec2.internal,
51048),rdd_206_1476,StorageLevel(false, true, false, false, 1),2636322,0,0)]
````
Which would happen right after a GC. Which lead me to check the [spark akka
settings](http://spark.apache.org/docs/1.2.1/configuration.html#networking).
And led me to me not finding *akka.remote.transport-failure-detector.threshold*
in akka.
Note that I noticed spark is using a [spark-specific version of
akka-actor](http://mvnrepository.com/artifact/org.spark-project.akka/akka-actor_2.10/2.3.4-spark),
and that might make this fix irrelevant/incorrect.
The relevant extract of the akka configuration page:
````
# Settings for the Phi accrual failure detector
(http://ddg.jaist.ac.jp/pub/HDY+04.pdf
# [Hayashibara et al]) used for remote death watch.
watch-failure-detector {
# FQCN of the failure detector implementation.
# It must implement akka.remote.FailureDetector and have
# a public constructor with a com.typesafe.config.Config and
# akka.actor.EventStream parameter.
implementation-class = "akka.remote.PhiAccrualFailureDetector"
# How often keep-alive heartbeat messages should be sent to each
connection.
heartbeat-interval = 1 s
# Defines the failure detector threshold.
# A low threshold is prone to generate many wrong suspicions but
ensures
# a quick detection in the event of a real crash. Conversely, a high
# threshold generates fewer mistakes but needs more time to detect
# actual crashes.
threshold = 10.0
# Number of the samples of inter-heartbeat arrival times to adaptively
# calculate the failure timeout for connections.
max-sample-size = 200
# Minimum standard deviation to use for the normal distribution in
# AccrualFailureDetector. Too low standard deviation might result in
# too much sensitivity for sudden, but normal, deviations in heartbeat
# inter arrival times.
min-std-deviation = 100 ms
# Number of potentially lost/delayed heartbeats that will be
# accepted before considering it to be an anomaly.
# This margin is important to be able to survive sudden, occasional,
# pauses in heartbeat arrivals, due to for example garbage collect or
# network drop.
acceptable-heartbeat-pause = 10 s
# How often to check for nodes marked as unreachable by the failure
# detector
unreachable-nodes-reaper-interval = 1s
# After the heartbeat request has been sent the first failure
detection
# will start after this period, even though no heartbeat mesage has
# been received.
expected-response-after = 3 s
}
````
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/thomas-gerber/spark master
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/spark/pull/4541.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #4541
----
commit eaf98963ec6a55593329e7aad85ed3d7e848f769
Author: Thomas Gerber <[email protected]>
Date: 2015-02-11T22:25:44Z
Fixed akka failure detection setting mappings
----
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]