Repository: flink
Updated Branches:
  refs/heads/master 8b3cbb525 -> de45c06f1


[docs] Adds akka configuration description

This closes #527.


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

Branch: refs/heads/master
Commit: de45c06f11198a08d513f54e8c5646191213bd21
Parents: 8b3cbb5
Author: Till Rohrmann <trohrm...@apache.org>
Authored: Thu Feb 19 18:01:42 2015 +0100
Committer: Till Rohrmann <trohrm...@apache.org>
Committed: Tue Mar 24 15:20:16 2015 +0100

----------------------------------------------------------------------
 docs/config.md                                  | 27 ++++++++++----------
 .../flink/configuration/ConfigConstants.java    |  3 +++
 .../apache/flink/runtime/akka/AkkaUtils.scala   |  7 ++---
 3 files changed, 18 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/de45c06f/docs/config.md
----------------------------------------------------------------------
diff --git a/docs/config.md b/docs/config.md
index 3524a60..251ce03 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -234,20 +234,19 @@ TaskManager hearbeat may be missing before the 
TaskManager is considered failed.
 
 ### Distributed Coordination (via Akka)
 
-- `akka.startup-timeout`: 
-- `akka.transport.heartbeat.interval`: 
-- `akka.transport.heartbeat.pause`: 
-- `akka.transport.threshold`: 
-- `akka.watch.heartbeat.interval`: 
-- `akka.watch.heartbeat.pause`: 
-- `akka.watch.threshold`: 
-- `akka.tcp.timeout`: 
-- `akka.framesize`: 
-- `akka.throughput`: 
-- `akka.log.lifecycle.events`: 
-- `akka.loglevel`: 
-- `akka.ask.timeout`: 
-
+- `akka.ask.timeout`: Timeout used for all futures and blocking Akka calls. If 
Flink fails due to timeouts then you should try to increase this value. 
Timeouts can be caused by slow machines or a congested network. The timeout 
value requires a time-unit specifier (ms/s/min/h/d) (DEFAULT: **100 s**).
+- `akka.lookup.timeout`: Timeout used for the lookup of the JobManager. The 
timeout value has to contain a time-unit specifier (ms/s/min/h/d) (DEFAULT: 
**10 s**).
+- `akka.framesize`: Maximum size of messages which are sent between the 
JobManager and the TaskManagers. If Flink fails because messages exceed this 
limit, then you should increase it. The message size requires a size-unit 
specifier (DEFAULT: **10485760b**).
+- `akka.watch.heartbeat.interval`: Heartbeat interval for Akka's DeathWatch 
mechanism to detect dead TaskManagers. If TaskManagers are wrongly marked dead 
because of lost or delayed heartbeat messages, then you should increase this 
value. A thorough description of Akka's DeathWatch can be found 
[here](http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector)
 (DEFAULT: **akka.ask.timeout/10**).
+- `akka.watch.heartbeat.pause`: Acceptable heartbeat pause for Akka's 
DeathWatch mechanism. A low value does not allow a irregular heartbeat. A 
thorough description of Akka's DeathWatch can be found 
[here](http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector)
 (DEFAULT: **akka.ask.timeout**).
+- `akka.watch.threshold`: Threshold for the DeathWatch failure detector. A low 
value is prone to false positives whereas a high value increases the time to 
detect a dead TaskManager. A thorough description of Akka's DeathWatch can be 
found 
[here](http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector)
 (DEFAULT: **12**).
+- `akka.transport.heartbeat.interval`: Heartbeat interval for Akka's transport 
failure detector. Since Flink uses TCP, the detector is not necessary. 
Therefore, the detector is disabled by setting the interval to a very high 
value. In case you should need the transport failure detector, set the interval 
to some reasonable value. The interval value requires a time-unit specifier 
(ms/s/min/h/d) (DEFAULT: **1000 s**).
+- `akka.transport.heartbeat.pause`: Acceptable heartbeat pause for Akka's 
transport failure detector. Since Flink uses TCP, the detector is not 
necessary. Therefore, the detector is disabled by setting the pause to a very 
high value. In case you should need the transport failure detector, set the 
pause to some reasonable value. The pause value requires a time-unit specifier 
(ms/s/min/h/d) (DEFAULT: **6000 s**).
+- `akka.transport.threshold`: Threshold for the transport failure detector. 
Since Flink uses TCP, the detector is not necessary and, thus, the threshold is 
set to a high value (DEFAULT: **300**).
+- `akka.tcp.timeout`: Timeout for all outbound connections. If you should 
experience problems with connecting to a TaskManager due to a slow network, you 
should increase this value (DEFAULT: **akka.ask.timeout**).
+- `akka.throughput`: Number of messages that are processed in a batch before 
returning the thread to the pool. Low values denote a fair scheduling whereas 
high values can increase the performance at the cost of unfairness (DEFAULT: 
**15**).
+- `akka.log.lifecycle.events`: Turns on the Akka's remote logging of events. 
Set this value to 'on' in case of debugging (DEFAULT: **off**).
+- `akka.startup-timeout`: Timeout after which the startup of a remote 
component is considered being failed (DEFAULT: **akka.ask.timeout**).
 
 ### JobManager Web Frontend
 

http://git-wip-us.apache.org/repos/asf/flink/blob/de45c06f/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java 
b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 09f55fd..44f146d 100644
--- 
a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ 
b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -328,6 +328,9 @@ public final class ConfigConstants {
 
        // ------------------------------ AKKA 
------------------------------------
 
+       /**
+        * Timeout for the startup of the actor system
+        */
        public static final String AKKA_STARTUP_TIMEOUT = 
"akka.startup-timeout";
 
        /**

http://git-wip-us.apache.org/repos/asf/flink/blob/de45c06f/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
----------------------------------------------------------------------
diff --git 
a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala 
b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
index be55e14..423cdfd 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
@@ -18,18 +18,15 @@
 
 package org.apache.flink.runtime.akka
 
-import java.io.IOException
 import java.net.InetAddress
 import java.util.concurrent.{TimeUnit, Callable}
 
-import akka.actor.Actor.Receive
 import akka.actor._
-import akka.pattern.{Patterns, ask => akkaAsk}
-import akka.remote.{RemotingLifecycleEvent, AssociationEvent}
+import akka.pattern.{ask => akkaAsk}
 import com.typesafe.config.{Config, ConfigFactory}
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
 import org.slf4j.LoggerFactory
-import scala.concurrent.{ExecutionContext, Future, Await}
+import scala.concurrent.{ExecutionContext, Future}
 import scala.concurrent.duration._
 import scala.language.postfixOps
 

Reply via email to